HADOOP-19315. Upgrade Apache Avro to 1.11.4 (#7128)

* All field access is now via setter/getter methods
* To use Avro to marshal Serializable objects,
  the packages they are in must be declared in the system property
  "org.apache.avro.SERIALIZABLE_PACKAGES"
  
This is required to address
- CVE-2024-47561
- CVE-2023-39410  

This change is not backwards compatible.

Contributed by Dominik Diedrich
This commit is contained in:
Dominik Diedrich 2024-11-11 16:46:36 +01:00 committed by GitHub
parent 9657276492
commit 9a743bd17f
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
12 changed files with 93 additions and 28 deletions

View File

@ -297,7 +297,7 @@ io.swagger:swagger-annotations:1.5.4
javax.inject:javax.inject:1 javax.inject:javax.inject:1
net.java.dev.jna:jna:5.2.0 net.java.dev.jna:jna:5.2.0
net.minidev:accessors-smart:1.2 net.minidev:accessors-smart:1.2
org.apache.avro:avro:1.9.2 org.apache.avro:avro:1.11.4
org.apache.avro:avro:1.11.3 org.apache.avro:avro:1.11.3
org.apache.commons:commons-compress:1.26.1 org.apache.commons:commons-compress:1.26.1
org.apache.commons:commons-configuration2:2.10.1 org.apache.commons:commons-configuration2:2.10.1

View File

@ -0,0 +1,35 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.constants;
/**
* Evolving config constants class used in various hadoop tests.
*/
public final class ConfigConstants {
private ConfigConstants() {}
/**
* System property name for the avro dependency.
* This property is used to configure trusted packages,
* which the avro dependency can use for serialization.
*/
public static final String CONFIG_AVRO_SERIALIZABLE_PACKAGES =
"org.apache.avro.SERIALIZABLE_PACKAGES";
}

View File

@ -0,0 +1,22 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
/**
* Evolving config constants class used in various hadoop tests.
*/
package org.apache.hadoop.constants;

View File

@ -17,6 +17,7 @@
*/ */
package org.apache.hadoop.fs; package org.apache.hadoop.fs;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
@ -30,6 +31,7 @@
import java.util.Arrays; import java.util.Arrays;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.constants.ConfigConstants;
import org.apache.hadoop.io.AvroTestUtil; import org.apache.hadoop.io.AvroTestUtil;
import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.util.Shell; import org.apache.hadoop.util.Shell;
@ -404,6 +406,8 @@ public void testGetName() {
@Test (timeout = 30000) @Test (timeout = 30000)
public void testAvroReflect() throws Exception { public void testAvroReflect() throws Exception {
// Avro expects explicitely stated, trusted packages used for (de-)serialization
System.setProperty(ConfigConstants.CONFIG_AVRO_SERIALIZABLE_PACKAGES, "org.apache.hadoop.fs");
AvroTestUtil.testReflect AvroTestUtil.testReflect
(new Path("foo"), (new Path("foo"),
"{\"type\":\"string\",\"java-class\":\"org.apache.hadoop.fs.Path\"}"); "{\"type\":\"string\",\"java-class\":\"org.apache.hadoop.fs.Path\"}");

View File

@ -24,6 +24,8 @@
import java.nio.charset.CharacterCodingException; import java.nio.charset.CharacterCodingException;
import java.nio.charset.StandardCharsets; import java.nio.charset.StandardCharsets;
import java.util.Random; import java.util.Random;
import org.apache.hadoop.constants.ConfigConstants;
import org.apache.hadoop.thirdparty.com.google.common.primitives.Bytes; import org.apache.hadoop.thirdparty.com.google.common.primitives.Bytes;
import org.junit.Test; import org.junit.Test;
@ -344,6 +346,8 @@ public void testConcurrentEncodeDecode() throws Exception{
@Test @Test
public void testAvroReflect() throws Exception { public void testAvroReflect() throws Exception {
// Avro expects explicitely stated, trusted packages used for (de-)serialization
System.setProperty(ConfigConstants.CONFIG_AVRO_SERIALIZABLE_PACKAGES, "org.apache.hadoop.io");
AvroTestUtil.testReflect AvroTestUtil.testReflect
(new Text("foo"), (new Text("foo"),
"{\"type\":\"string\",\"java-class\":\"org.apache.hadoop.io.Text\"}"); "{\"type\":\"string\",\"java-class\":\"org.apache.hadoop.io.Text\"}");

View File

@ -33,7 +33,7 @@ public class TestAvroSerialization {
@Test @Test
public void testSpecific() throws Exception { public void testSpecific() throws Exception {
AvroRecord before = new AvroRecord(); AvroRecord before = new AvroRecord();
before.intField = 5; before.setIntField(5);
AvroRecord after = SerializationTestUtil.testSerialization(conf, before); AvroRecord after = SerializationTestUtil.testSerialization(conf, before);
assertEquals(before, after); assertEquals(before, after);
} }

View File

@ -20,7 +20,6 @@
import java.util.Set; import java.util.Set;
import org.apache.avro.util.Utf8;
import org.apache.hadoop.mapreduce.JobID; import org.apache.hadoop.mapreduce.JobID;
import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent;
@ -31,8 +30,8 @@ public class JobQueueChangeEvent implements HistoryEvent {
private JobQueueChange datum = new JobQueueChange(); private JobQueueChange datum = new JobQueueChange();
public JobQueueChangeEvent(JobID id, String queueName) { public JobQueueChangeEvent(JobID id, String queueName) {
datum.jobid = new Utf8(id.toString()); datum.setJobid(id.toString());
datum.jobQueueName = new Utf8(queueName); datum.setJobQueueName(queueName);
} }
JobQueueChangeEvent() { } JobQueueChangeEvent() { }
@ -54,13 +53,14 @@ public void setDatum(Object datum) {
/** Get the Job ID */ /** Get the Job ID */
public JobID getJobId() { public JobID getJobId() {
return JobID.forName(datum.jobid.toString()); return JobID.forName(datum.getJobid().toString());
} }
/** Get the new Job queue name */ /** Get the new Job queue name */
public String getJobQueueName() { public String getJobQueueName() {
if (datum.jobQueueName != null) { java.lang.CharSequence jobQueueName = datum.getJobQueueName();
return datum.jobQueueName.toString(); if (jobQueueName != null) {
return jobQueueName.toString();
} }
return null; return null;
} }

View File

@ -63,7 +63,7 @@
<java.security.egd>file:///dev/urandom</java.security.egd> <java.security.egd>file:///dev/urandom</java.security.egd>
<!-- avro version --> <!-- avro version -->
<avro.version>1.9.2</avro.version> <avro.version>1.11.4</avro.version>
<!-- jersey version --> <!-- jersey version -->
<jersey.version>1.19.4</jersey.version> <jersey.version>1.19.4</jersey.version>

View File

@ -460,7 +460,7 @@ private void processTaskFinishedEvent(TaskFinishedEvent event) {
} }
task.setFinishTime(event.getFinishTime()); task.setFinishTime(event.getFinishTime());
task.setTaskStatus(getPre21Value(event.getTaskStatus())); task.setTaskStatus(getPre21Value(event.getTaskStatus()));
task.incorporateCounters(((TaskFinished) event.getDatum()).counters); task.incorporateCounters(((TaskFinished) event.getDatum()).getCounters());
} }
private void processTaskFailedEvent(TaskFailedEvent event) { private void processTaskFailedEvent(TaskFailedEvent event) {
@ -472,7 +472,7 @@ private void processTaskFailedEvent(TaskFailedEvent event) {
task.setFinishTime(event.getFinishTime()); task.setFinishTime(event.getFinishTime());
task.setTaskStatus(getPre21Value(event.getTaskStatus())); task.setTaskStatus(getPre21Value(event.getTaskStatus()));
TaskFailed t = (TaskFailed)(event.getDatum()); TaskFailed t = (TaskFailed)(event.getDatum());
task.putDiagnosticInfo(t.error.toString()); task.putDiagnosticInfo(t.getError().toString());
// killed task wouldn't have failed attempt. // killed task wouldn't have failed attempt.
if (t.getFailedDueToAttempt() != null) { if (t.getFailedDueToAttempt() != null) {
task.putFailedDueToAttemptId(t.getFailedDueToAttempt().toString()); task.putFailedDueToAttemptId(t.getFailedDueToAttempt().toString());
@ -542,7 +542,7 @@ private void processTaskAttemptFinishedEvent(TaskAttemptFinishedEvent event) {
} }
attempt.setFinishTime(event.getFinishTime()); attempt.setFinishTime(event.getFinishTime());
attempt attempt
.incorporateCounters(((TaskAttemptFinished) event.getDatum()).counters); .incorporateCounters(((TaskAttemptFinished) event.getDatum()).getCounters());
} }
private void processReduceAttemptFinishedEvent( private void processReduceAttemptFinishedEvent(
@ -568,7 +568,7 @@ private void processReduceAttemptFinishedEvent(
attempt.setShuffleFinished(event.getShuffleFinishTime()); attempt.setShuffleFinished(event.getShuffleFinishTime());
attempt.setSortFinished(event.getSortFinishTime()); attempt.setSortFinished(event.getSortFinishTime());
attempt attempt
.incorporateCounters(((ReduceAttemptFinished) event.getDatum()).counters); .incorporateCounters(((ReduceAttemptFinished) event.getDatum()).getCounters());
attempt.arraySetClockSplits(event.getClockSplits()); attempt.arraySetClockSplits(event.getClockSplits());
attempt.arraySetCpuUsages(event.getCpuUsages()); attempt.arraySetCpuUsages(event.getCpuUsages());
attempt.arraySetVMemKbytes(event.getVMemKbytes()); attempt.arraySetVMemKbytes(event.getVMemKbytes());
@ -596,7 +596,7 @@ private void processMapAttemptFinishedEvent(MapAttemptFinishedEvent event) {
// is redundant, but making this will add future-proofing. // is redundant, but making this will add future-proofing.
attempt.setFinishTime(event.getFinishTime()); attempt.setFinishTime(event.getFinishTime());
attempt attempt
.incorporateCounters(((MapAttemptFinished) event.getDatum()).counters); .incorporateCounters(((MapAttemptFinished) event.getDatum()).getCounters());
attempt.arraySetClockSplits(event.getClockSplits()); attempt.arraySetClockSplits(event.getClockSplits());
attempt.arraySetCpuUsages(event.getCpuUsages()); attempt.arraySetCpuUsages(event.getCpuUsages());
attempt.arraySetVMemKbytes(event.getVMemKbytes()); attempt.arraySetVMemKbytes(event.getVMemKbytes());
@ -661,11 +661,11 @@ private void processJobFinishedEvent(JobFinishedEvent event) {
JobFinished job = (JobFinished)event.getDatum(); JobFinished job = (JobFinished)event.getDatum();
Map<String, Long> countersMap = Map<String, Long> countersMap =
JobHistoryUtils.extractCounters(job.totalCounters); JobHistoryUtils.extractCounters(job.getTotalCounters());
result.putTotalCounters(countersMap); result.putTotalCounters(countersMap);
countersMap = JobHistoryUtils.extractCounters(job.mapCounters); countersMap = JobHistoryUtils.extractCounters(job.getMapCounters());
result.putMapCounters(countersMap); result.putMapCounters(countersMap);
countersMap = JobHistoryUtils.extractCounters(job.reduceCounters); countersMap = JobHistoryUtils.extractCounters(job.getReduceCounters());
result.putReduceCounters(countersMap); result.putReduceCounters(countersMap);
} }

View File

@ -157,9 +157,9 @@ static boolean isJobConfXml(String fileName) {
static Map<String, Long> extractCounters(JhCounters counters) { static Map<String, Long> extractCounters(JhCounters counters) {
Map<String, Long> countersMap = new HashMap<String, Long>(); Map<String, Long> countersMap = new HashMap<String, Long>();
if (counters != null) { if (counters != null) {
for (JhCounterGroup group : counters.groups) { for (JhCounterGroup group : counters.getGroups()) {
for (JhCounter counter : group.counts) { for (JhCounter counter : group.getCounts()) {
countersMap.put(counter.name.toString(), counter.value); countersMap.put(counter.getName().toString(), counter.getValue());
} }
} }
} }

View File

@ -268,11 +268,11 @@ private static void incorporateCounter(SetField thunk, JhCounters counters,
String counterName) { String counterName) {
counterName = canonicalizeCounterName(counterName); counterName = canonicalizeCounterName(counterName);
for (JhCounterGroup group : counters.groups) { for (JhCounterGroup group : counters.getGroups()) {
for (JhCounter counter : group.counts) { for (JhCounter counter : group.getCounts()) {
if (counterName if (counterName
.equals(canonicalizeCounterName(counter.name.toString()))) { .equals(canonicalizeCounterName(counter.getName().toString()))) {
thunk.set(counter.value); thunk.set(counter.getValue());
return; return;
} }
} }

View File

@ -636,11 +636,11 @@ private static void incorporateCounter(SetField thunk, JhCounters counters,
String counterName) { String counterName) {
counterName = canonicalizeCounterName(counterName); counterName = canonicalizeCounterName(counterName);
for (JhCounterGroup group : counters.groups) { for (JhCounterGroup group : counters.getGroups()) {
for (JhCounter counter : group.counts) { for (JhCounter counter : group.getCounts()) {
if (counterName if (counterName
.equals(canonicalizeCounterName(counter.name.toString()))) { .equals(canonicalizeCounterName(counter.getName().toString()))) {
thunk.set(counter.value); thunk.set(counter.getValue());
return; return;
} }
} }