none, gzip, snappy, lz4, or zstd. "
                                                        + "Compression is of full batches of data, so the efficacy of batching will also impact the compression ratio (more batching means better compression).";
 
+    /** compression.gzip.level */
+    public static final String COMPRESSION_GZIP_LEVEL_CONFIG = "compression.gzip.level";
+    private static final String COMPRESSION_GZIP_LEVEL_DOC = "The compression level to use if " + COMPRESSION_TYPE_CONFIG + " is set to gzip.";
+
+    /** compression.lz4.level */
+    public static final String COMPRESSION_LZ4_LEVEL_CONFIG = "compression.lz4.level";
+    private static final String COMPRESSION_LZ4_LEVEL_DOC = "The compression level to use if " + COMPRESSION_TYPE_CONFIG + " is set to lz4.";
+
+    /** compression.zstd.level */
+    public static final String COMPRESSION_ZSTD_LEVEL_CONFIG = "compression.zstd.level";
+    private static final String COMPRESSION_ZSTD_LEVEL_DOC = "The compression level to use if " + COMPRESSION_TYPE_CONFIG + " is set to zstd.";
+
     /** metrics.sample.window.ms */
     public static final String METRICS_SAMPLE_WINDOW_MS_CONFIG = CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_CONFIG;
 
@@ -364,6 +379,9 @@ public class ProducerConfig extends AbstractConfig {
                                         Importance.LOW,
                                         ACKS_DOC)
                                 .define(COMPRESSION_TYPE_CONFIG, Type.STRING, CompressionType.NONE.name, in(Utils.enumOptions(CompressionType.class)), Importance.HIGH, COMPRESSION_TYPE_DOC)
+                                .define(COMPRESSION_GZIP_LEVEL_CONFIG, Type.INT, GzipCompression.DEFAULT_LEVEL, new GzipCompression.LevelValidator(), Importance.MEDIUM, COMPRESSION_GZIP_LEVEL_DOC)
+                                .define(COMPRESSION_LZ4_LEVEL_CONFIG, Type.INT, Lz4Compression.DEFAULT_LEVEL, between(Lz4Compression.MIN_LEVEL, Lz4Compression.MAX_LEVEL), Importance.MEDIUM, COMPRESSION_LZ4_LEVEL_DOC)
+                                .define(COMPRESSION_ZSTD_LEVEL_CONFIG, Type.INT, ZstdCompression.DEFAULT_LEVEL, between(ZstdCompression.MIN_LEVEL, ZstdCompression.MAX_LEVEL), Importance.MEDIUM, COMPRESSION_ZSTD_LEVEL_DOC)
                                 .define(BATCH_SIZE_CONFIG, Type.INT, 16384, atLeast(0), Importance.MEDIUM, BATCH_SIZE_DOC)
                                 .define(PARTITIONER_ADPATIVE_PARTITIONING_ENABLE_CONFIG, Type.BOOLEAN, true, Importance.LOW, PARTITIONER_ADPATIVE_PARTITIONING_ENABLE_DOC)
                                 .define(PARTITIONER_AVAILABILITY_TIMEOUT_MS_CONFIG, Type.LONG, 0, atLeast(0), Importance.LOW, PARTITIONER_AVAILABILITY_TIMEOUT_MS_DOC)
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerBatch.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerBatch.java
index 4225b428093..d21c3fa732d 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerBatch.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerBatch.java
@@ -99,7 +99,7 @@ public final class ProducerBatch {
         this.retry = false;
         this.isSplitBatch = isSplitBatch;
         float compressionRatioEstimation = CompressionRatioEstimator.estimation(topicPartition.topic(),
-                                                                                recordsBuilder.compressionType());
+                                                                                recordsBuilder.compression().type());
         this.currentLeaderEpoch = OptionalInt.empty();
         this.attemptsWhenLeaderLastChanged = 0;
         recordsBuilder.setEstimatedCompressionRatio(compressionRatioEstimation);
@@ -146,7 +146,7 @@ public final class ProducerBatch {
         } else {
             this.recordsBuilder.append(timestamp, key, value, headers);
             this.maxRecordSize = Math.max(this.maxRecordSize, AbstractRecords.estimateSizeInBytesUpperBound(magic(),
-                    recordsBuilder.compressionType(), key, value, headers));
+                    recordsBuilder.compression().type(), key, value, headers));
             this.lastAppendTime = now;
             FutureRecordMetadata future = new FutureRecordMetadata(this.produceFuture, this.recordCount,
                                                                    timestamp,
@@ -172,7 +172,7 @@ public final class ProducerBatch {
             // No need to get the CRC.
             this.recordsBuilder.append(timestamp, key, value, headers);
             this.maxRecordSize = Math.max(this.maxRecordSize, AbstractRecords.estimateSizeInBytesUpperBound(magic(),
-                    recordsBuilder.compressionType(), key, value, headers));
+                    recordsBuilder.compression().type(), key, value, headers));
             FutureRecordMetadata future = new FutureRecordMetadata(this.produceFuture, this.recordCount,
                                                                    timestamp,
                                                                    key == null ? -1 : key.remaining(),
@@ -377,19 +377,19 @@ public final class ProducerBatch {
 
     private ProducerBatch createBatchOffAccumulatorForRecord(Record record, int batchSize) {
         int initialSize = Math.max(AbstractRecords.estimateSizeInBytesUpperBound(magic(),
-                recordsBuilder.compressionType(), record.key(), record.value(), record.headers()), batchSize);
+                recordsBuilder.compression().type(), record.key(), record.value(), record.headers()), batchSize);
         ByteBuffer buffer = ByteBuffer.allocate(initialSize);
 
         // Note that we intentionally do not set producer state (producerId, epoch, sequence, and isTransactional)
         // for the newly created batch. This will be set when the batch is dequeued for sending (which is consistent
         // with how normal batches are handled).
-        MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, magic(), recordsBuilder.compressionType(),
+        MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, magic(), recordsBuilder.compression(),
                 TimestampType.CREATE_TIME, 0L);
         return new ProducerBatch(topicPartition, builder, this.createdMs, true);
     }
 
     public boolean isCompressed() {
-        return recordsBuilder.compressionType() != CompressionType.NONE;
+        return recordsBuilder.compression().type() != CompressionType.NONE;
     }
 
     /**
@@ -491,7 +491,7 @@ public final class ProducerBatch {
         recordsBuilder.close();
         if (!recordsBuilder.isControlBatch()) {
             CompressionRatioEstimator.updateEstimation(topicPartition.topic(),
-                                                       recordsBuilder.compressionType(),
+                                                       recordsBuilder.compression().type(),
                                                        (float) recordsBuilder.compressionRatio());
         }
         reopened = false;
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java
index 013ad32dc7d..12e77b0d516 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java
@@ -36,6 +36,7 @@ import org.apache.kafka.clients.CommonClientConfigs;
 import org.apache.kafka.clients.MetadataSnapshot;
 import org.apache.kafka.clients.producer.Callback;
 import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.common.compress.Compression;
 import org.apache.kafka.common.utils.ExponentialBackoff;
 import org.apache.kafka.common.utils.ProducerIdAndEpoch;
 import org.apache.kafka.common.Cluster;
@@ -48,7 +49,6 @@ import org.apache.kafka.common.header.Header;
 import org.apache.kafka.common.metrics.Metrics;
 import org.apache.kafka.common.record.AbstractRecords;
 import org.apache.kafka.common.record.CompressionRatioEstimator;
-import org.apache.kafka.common.record.CompressionType;
 import org.apache.kafka.common.record.MemoryRecords;
 import org.apache.kafka.common.record.MemoryRecordsBuilder;
 import org.apache.kafka.common.record.Record;
@@ -74,7 +74,7 @@ public class RecordAccumulator {
     private final AtomicInteger flushesInProgress;
     private final AtomicInteger appendsInProgress;
     private final int batchSize;
-    private final CompressionType compression;
+    private final Compression compression;
     private final int lingerMs;
     private final ExponentialBackoff retryBackoff;
     private final int deliveryTimeoutMs;
@@ -116,7 +116,7 @@ public class RecordAccumulator {
      */
     public RecordAccumulator(LogContext logContext,
                              int batchSize,
-                             CompressionType compression,
+                             Compression compression,
                              int lingerMs,
                              long retryBackoffMs,
                              long retryBackoffMaxMs,
@@ -176,7 +176,7 @@ public class RecordAccumulator {
      */
     public RecordAccumulator(LogContext logContext,
                              int batchSize,
-                             CompressionType compression,
+                             Compression compression,
                              int lingerMs,
                              long retryBackoffMs,
                              long retryBackoffMaxMs,
@@ -344,7 +344,7 @@ public class RecordAccumulator {
 
                 if (buffer == null) {
                     byte maxUsableMagic = apiVersions.maxUsableProduceMagic();
-                    int size = Math.max(this.batchSize, AbstractRecords.estimateSizeInBytesUpperBound(maxUsableMagic, compression, key, value, headers));
+                    int size = Math.max(this.batchSize, AbstractRecords.estimateSizeInBytesUpperBound(maxUsableMagic, compression.type(), key, value, headers));
                     log.trace("Allocating a new {} byte message buffer for topic {} partition {} with remaining timeout {}ms", size, topic, partition, maxTimeToBlock);
                     // This call may block if we exhausted buffer space.
                     buffer = free.allocate(size, maxTimeToBlock);
@@ -533,7 +533,7 @@ public class RecordAccumulator {
         // Reset the estimated compression ratio to the initial value or the big batch compression ratio, whichever
         // is bigger. There are several different ways to do the reset. We chose the most conservative one to ensure
         // the split doesn't happen too often.
-        CompressionRatioEstimator.setEstimation(bigBatch.topicPartition.topic(), compression,
+        CompressionRatioEstimator.setEstimation(bigBatch.topicPartition.topic(), compression.type(),
                                                 Math.max(1.0f, (float) bigBatch.compressionRatio()));
         Dequegzip.";
+    public static final String COMPRESSION_LZ4_LEVEL_CONFIG = "compression.lz4.level";
+    public static final String COMPRESSION_LZ4_LEVEL_DOC = "The compression level to use if " + COMPRESSION_TYPE_CONFIG + " is set to lz4.";
+    public static final String COMPRESSION_ZSTD_LEVEL_CONFIG = "compression.zstd.level";
+    public static final String COMPRESSION_ZSTD_LEVEL_DOC = "The compression level to use if " + COMPRESSION_TYPE_CONFIG + " is set to zstd.";
+
     public static final String PREALLOCATE_CONFIG = "preallocate";
     public static final String PREALLOCATE_DOC = "True if we should preallocate the file on disk when " +
         "creating a new log segment.";
diff --git a/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java b/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java
index 499dc50289c..0f295816237 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/AbstractLegacyRecordBatch.java
@@ -18,6 +18,7 @@ package org.apache.kafka.common.record;
 
 import org.apache.kafka.common.InvalidRecordException;
 import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.compress.Compression;
 import org.apache.kafka.common.errors.CorruptRecordException;
 import org.apache.kafka.common.header.Header;
 import org.apache.kafka.common.utils.AbstractIterator;
@@ -332,7 +333,7 @@ public abstract class AbstractLegacyRecordBatch extends AbstractRecordBatch impl
                 throw new InvalidRecordException("Found invalid compressed record set with null value (magic = " +
                         wrapperMagic + ")");
 
-            InputStream stream = compressionType.wrapForInput(wrapperValue, wrapperRecord.magic(), bufferSupplier);
+            InputStream stream = Compression.of(compressionType).build().wrapForInput(wrapperValue, wrapperRecord.magic(), bufferSupplier);
             LogInputStream
-     * Note: Unlike {@link #wrapForInput}, {@link #wrapForOutput} cannot take {@link ByteBuffer}s directly.
-     * Currently, {@link MemoryRecordsBuilder#writeDefaultBatchHeader()} and {@link MemoryRecordsBuilder#writeLegacyCompressedWrapperHeader()}
-     * write to the underlying buffer in the given {@link ByteBufferOutputStream} after the compressed data has been written.
-     * In the event that the buffer needs to be expanded while writing the data, access to the underlying buffer needs to be preserved.
-     */
-    public abstract OutputStream wrapForOutput(ByteBufferOutputStream bufferStream, byte messageVersion);
-
-    /**
-     * Wrap buffer with an InputStream that will decompress data with this CompressionType.
-     *
-     * @param decompressionBufferSupplier The supplier of ByteBuffer(s) used for decompression if supported.
-     *                                    For small record batches, allocating a potentially large buffer (64 KB for LZ4)
-     *                                    will dominate the cost of decompressing and iterating over the records in the
-     *                                    batch. As such, a supplier that reuses buffers will have a significant
-     *                                    performance impact.
-     */
-    public abstract InputStream wrapForInput(ByteBuffer buffer, byte messageVersion, BufferSupplier decompressionBufferSupplier);
-
-    /**
-     * Recommended size of buffer for storing decompressed output.
-     */
-    public int decompressionOutputSize() {
-        throw new UnsupportedOperationException("Size of decompression buffer is not defined for this compression type=" + this.name);
-    }
-
     public static CompressionType forId(int id) {
         switch (id) {
             case 0:
diff --git a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java
index b1b8a2ad6a9..27629330d7a 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/DefaultRecordBatch.java
@@ -18,6 +18,7 @@ package org.apache.kafka.common.record;
 
 import org.apache.kafka.common.InvalidRecordException;
 import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.compress.Compression;
 import org.apache.kafka.common.errors.CorruptRecordException;
 import org.apache.kafka.common.header.Header;
 import org.apache.kafka.common.utils.BufferSupplier;
@@ -270,7 +271,7 @@ public class DefaultRecordBatch extends AbstractRecordBatch implements MutableRe
     public InputStream recordInputStream(BufferSupplier bufferSupplier) {
         final ByteBuffer buffer = this.buffer.duplicate();
         buffer.position(RECORDS_OFFSET);
-        return compressionType().wrapForInput(buffer, magic(), bufferSupplier);
+        return Compression.of(compressionType()).build().wrapForInput(buffer, magic(), bufferSupplier);
     }
 
     private CloseableIterator${classOf[Authorizer].getName}" +
@@ -185,7 +192,6 @@ object KafkaConfig {
   /** ********* Request Limit Configuration **************/
   val MaxRequestPartitionSizeLimitDoc = "The maximum number of partitions can be served in one request."
 
-
   /** ********* Delegation Token Configuration ****************/
   val DelegationTokenSecretKeyAliasDoc = s"DEPRECATED: An alias for $DelegationTokenSecretKeyProp, which should be used instead of this config."
   val DelegationTokenSecretKeyDoc = "Secret key to generate and verify delegation tokens. The same key must be configured across all the brokers. " +
@@ -408,6 +414,9 @@ object KafkaConfig {
       .define(GroupCoordinatorConfig.OFFSET_COMMIT_REQUIRED_ACKS_CONFIG, SHORT, GroupCoordinatorConfig.OFFSET_COMMIT_REQUIRED_ACKS_DEFAULT, HIGH, GroupCoordinatorConfig.OFFSET_COMMIT_REQUIRED_ACKS_DOC)
       .define(DeleteTopicEnableProp, BOOLEAN, Defaults.DELETE_TOPIC_ENABLE, HIGH, DeleteTopicEnableDoc)
       .define(CompressionTypeProp, STRING, LogConfig.DEFAULT_COMPRESSION_TYPE, in(BrokerCompressionType.names.asScala.toSeq:_*), HIGH, CompressionTypeDoc)
+      .define(CompressionGzipLevelProp, INT, GzipCompression.DEFAULT_LEVEL, new GzipCompression.LevelValidator(), MEDIUM, CompressionGzipLevelDoc)
+      .define(CompressionLz4LevelProp, INT, Lz4Compression.DEFAULT_LEVEL, between(Lz4Compression.MIN_LEVEL, Lz4Compression.MAX_LEVEL), MEDIUM, CompressionLz4LevelDoc)
+      .define(CompressionZstdLevelProp, INT, ZstdCompression.DEFAULT_LEVEL, between(ZstdCompression.MIN_LEVEL, ZstdCompression.MAX_LEVEL), MEDIUM, CompressionZstdLevelDoc)
 
       /** ********* Transaction management configuration ***********/
       .define(TransactionStateManagerConfigs.TRANSACTIONAL_ID_EXPIRATION_MS_CONFIG, INT, TransactionStateManagerConfigs.TRANSACTIONAL_ID_EXPIRATION_MS_DEFAULT, atLeast(1), HIGH, TransactionStateManagerConfigs.TRANSACTIONAL_ID_EXPIRATION_MS_DOC)
@@ -1143,6 +1152,9 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
 
   val deleteTopicEnable = getBoolean(KafkaConfig.DeleteTopicEnableProp)
   def compressionType = getString(KafkaConfig.CompressionTypeProp)
+  def gzipCompressionLevel = getInt(KafkaConfig.CompressionGzipLevelProp)
+  def lz4CompressionLevel = getInt(KafkaConfig.CompressionLz4LevelProp)
+  def zstdCompressionLevel = getInt(KafkaConfig.CompressionZstdLevelProp)
 
   /** ********* Raft Quorum Configuration *********/
   val quorumVoters = getList(QuorumConfig.QUORUM_VOTERS_CONFIG)
@@ -1579,6 +1591,9 @@ class KafkaConfig private(doLog: Boolean, val props: java.util.Map[_, _], dynami
     logProps.put(TopicConfig.CLEANUP_POLICY_CONFIG, logCleanupPolicy)
     logProps.put(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, minInSyncReplicas)
     logProps.put(TopicConfig.COMPRESSION_TYPE_CONFIG, compressionType)
+    logProps.put(TopicConfig.COMPRESSION_GZIP_LEVEL_CONFIG, gzipCompressionLevel)
+    logProps.put(TopicConfig.COMPRESSION_LZ4_LEVEL_CONFIG, lz4CompressionLevel)
+    logProps.put(TopicConfig.COMPRESSION_ZSTD_LEVEL_CONFIG, zstdCompressionLevel)
     logProps.put(TopicConfig.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, uncleanLeaderElectionEnable)
     logProps.put(TopicConfig.PREALLOCATE_CONFIG, logPreAllocateEnable)
     logProps.put(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG, logMessageFormatVersion.version)
diff --git a/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java b/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java
index 85a3a3a8fa5..308212ff58d 100644
--- a/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java
+++ b/core/src/test/java/kafka/log/remote/RemoteLogManagerTest.java
@@ -27,10 +27,10 @@ import org.apache.kafka.common.KafkaException;
 import org.apache.kafka.common.TopicIdPartition;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.compress.Compression;
 import org.apache.kafka.common.config.AbstractConfig;
 import org.apache.kafka.common.errors.ReplicaNotAvailableException;
 import org.apache.kafka.common.network.ListenerName;
-import org.apache.kafka.common.record.CompressionType;
 import org.apache.kafka.common.record.FileRecords;
 import org.apache.kafka.common.record.MemoryRecords;
 import org.apache.kafka.common.record.RecordBatch;
@@ -1210,7 +1210,7 @@ public class RemoteLogManagerTest {
     private MemoryRecords records(long timestamp,
                                   long initialOffset,
                                   int partitionLeaderEpoch) {
-        return MemoryRecords.withRecords(initialOffset, CompressionType.NONE, partitionLeaderEpoch,
+        return MemoryRecords.withRecords(initialOffset, Compression.NONE, partitionLeaderEpoch,
             new SimpleRecord(timestamp - 1, "first message".getBytes()),
             new SimpleRecord(timestamp + 1, "second message".getBytes()),
             new SimpleRecord(timestamp + 2, "third message".getBytes())
diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
index d35b2c3e112..c00abf9427a 100644
--- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
+++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
@@ -26,6 +26,7 @@ import org.apache.kafka.clients.producer._
 import org.apache.kafka.common.acl.AclOperation._
 import org.apache.kafka.common.acl.AclPermissionType.{ALLOW, DENY}
 import org.apache.kafka.common.acl.{AccessControlEntry, AccessControlEntryFilter, AclBindingFilter, AclOperation, AclPermissionType}
+import org.apache.kafka.common.compress.Compression
 import org.apache.kafka.common.config.{ConfigResource, LogLevelConfig, TopicConfig}
 import org.apache.kafka.common.errors._
 import org.apache.kafka.common.internals.Topic.GROUP_METADATA_TOPIC_NAME
@@ -42,7 +43,7 @@ import org.apache.kafka.common.message.UpdateMetadataRequestData.{UpdateMetadata
 import org.apache.kafka.common.message.{AddOffsetsToTxnRequestData, AlterPartitionReassignmentsRequestData, AlterReplicaLogDirsRequestData, ControlledShutdownRequestData, CreateAclsRequestData, CreatePartitionsRequestData, CreateTopicsRequestData, DeleteAclsRequestData, DeleteGroupsRequestData, DeleteRecordsRequestData, DeleteTopicsRequestData, DescribeClusterRequestData, DescribeConfigsRequestData, DescribeGroupsRequestData, DescribeLogDirsRequestData, DescribeProducersRequestData, DescribeTransactionsRequestData, FindCoordinatorRequestData, HeartbeatRequestData, IncrementalAlterConfigsRequestData, JoinGroupRequestData, ListPartitionReassignmentsRequestData, ListTransactionsRequestData, MetadataRequestData, OffsetCommitRequestData, ProduceRequestData, SyncGroupRequestData, WriteTxnMarkersRequestData}
 import org.apache.kafka.common.network.ListenerName
 import org.apache.kafka.common.protocol.{ApiKeys, Errors}
-import org.apache.kafka.common.record.{CompressionType, MemoryRecords, RecordBatch, SimpleRecord}
+import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, SimpleRecord}
 import org.apache.kafka.common.requests.OffsetFetchResponse.PartitionData
 import org.apache.kafka.common.requests._
 import org.apache.kafka.common.resource.PatternType.{LITERAL, PREFIXED}
@@ -257,7 +258,7 @@ class AuthorizerIntegrationTest extends AbstractAuthorizerIntegrationTest {
           .setName(tp.topic).setPartitionData(Collections.singletonList(
           new ProduceRequestData.PartitionProduceData()
             .setIndex(tp.partition)
-            .setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("test".getBytes))))))
+            .setRecords(MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("test".getBytes))))))
         .iterator))
       .setAcks(1.toShort)
       .setTimeoutMs(5000))
diff --git a/core/src/test/scala/integration/kafka/network/DynamicConnectionQuotaTest.scala b/core/src/test/scala/integration/kafka/network/DynamicConnectionQuotaTest.scala
index 7b5f274d996..8efa1cd15ab 100644
--- a/core/src/test/scala/integration/kafka/network/DynamicConnectionQuotaTest.scala
+++ b/core/src/test/scala/integration/kafka/network/DynamicConnectionQuotaTest.scala
@@ -21,11 +21,12 @@ package kafka.network
 import kafka.server.BaseRequestTest
 import kafka.utils.TestUtils
 import org.apache.kafka.clients.admin.{Admin, AlterClientQuotasResult}
+import org.apache.kafka.common.compress.Compression
 import org.apache.kafka.common.message.ProduceRequestData
 import org.apache.kafka.common.network.ListenerName
 import org.apache.kafka.common.protocol.Errors
 import org.apache.kafka.common.quota.{ClientQuotaAlteration, ClientQuotaEntity}
-import org.apache.kafka.common.record.{CompressionType, MemoryRecords, SimpleRecord}
+import org.apache.kafka.common.record.{MemoryRecords, SimpleRecord}
 import org.apache.kafka.common.requests.{ProduceRequest, ProduceResponse}
 import org.apache.kafka.common.security.auth.SecurityProtocol
 import org.apache.kafka.common.{KafkaException, requests}
@@ -308,7 +309,7 @@ class DynamicConnectionQuotaTest extends BaseRequestTest {
           .setName(topic)
           .setPartitionData(Collections.singletonList(new ProduceRequestData.PartitionProduceData()
             .setIndex(0)
-            .setRecords(MemoryRecords.withRecords(CompressionType.NONE,
+            .setRecords(MemoryRecords.withRecords(Compression.NONE,
               new SimpleRecord(System.currentTimeMillis(), "key".getBytes, "value".getBytes))))))
         .iterator))
       .setAcks((-1).toShort)
diff --git a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala
index a827aca727d..379b454bc84 100644
--- a/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala
+++ b/core/src/test/scala/integration/kafka/server/DynamicBrokerReconfigurationTest.scala
@@ -681,7 +681,7 @@ class DynamicBrokerReconfigurationTest extends QuorumTestHarness with SaslSetup
     val log2 = servers.head.logManager.getLog(new TopicPartition(Topic.GROUP_METADATA_TOPIC_NAME, 0))
       .getOrElse(throw new IllegalStateException("Log not found"))
     assertFalse(log2.config.delete, "Overridden clean up policy should not be updated")
-    assertEquals(BrokerCompressionType.PRODUCER.name, log2.config.compressionType)
+    assertEquals(BrokerCompressionType.PRODUCER, log2.config.compressionType)
 
     // Verify that we can alter subset of log configs
     props.clear()
diff --git a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala
index 4d667499648..5bf8f328845 100644
--- a/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala
+++ b/core/src/test/scala/kafka/raft/KafkaMetadataLogTest.scala
@@ -19,10 +19,11 @@ package kafka.raft
 import kafka.log.UnifiedLog
 import kafka.server.{KafkaConfig, KafkaRaftServer}
 import kafka.utils.TestUtils
+import org.apache.kafka.common.compress.Compression
 import org.apache.kafka.common.errors.{InvalidConfigurationException, RecordTooLargeException}
 import org.apache.kafka.common.protocol
 import org.apache.kafka.common.protocol.{ObjectSerializationCache, Writable}
-import org.apache.kafka.common.record.{CompressionType, MemoryRecords, SimpleRecord}
+import org.apache.kafka.common.record.{MemoryRecords, SimpleRecord}
 import org.apache.kafka.common.utils.Utils
 import org.apache.kafka.raft._
 import org.apache.kafka.raft.internals.BatchBuilder
@@ -89,7 +90,7 @@ final class KafkaMetadataLogTest {
     val initialOffset = log.endOffset().offset
 
     log.appendAsLeader(
-      MemoryRecords.withRecords(initialOffset, CompressionType.NONE, currentEpoch, recordFoo),
+      MemoryRecords.withRecords(initialOffset, Compression.NONE, currentEpoch, recordFoo),
       currentEpoch
     )
 
@@ -98,7 +99,7 @@ final class KafkaMetadataLogTest {
       classOf[RuntimeException],
       () => {
         log.appendAsLeader(
-          MemoryRecords.withRecords(initialOffset, CompressionType.NONE, currentEpoch, recordFoo),
+          MemoryRecords.withRecords(initialOffset, Compression.NONE, currentEpoch, recordFoo),
           currentEpoch
         )
       }
@@ -108,7 +109,7 @@ final class KafkaMetadataLogTest {
       classOf[RuntimeException],
       () => {
         log.appendAsFollower(
-          MemoryRecords.withRecords(initialOffset, CompressionType.NONE, currentEpoch, recordFoo)
+          MemoryRecords.withRecords(initialOffset, Compression.NONE, currentEpoch, recordFoo)
         )
       }
     )
@@ -647,7 +648,7 @@ final class KafkaMetadataLogTest {
     val batchBuilder = new BatchBuilder[Array[Byte]](
       buffer,
       new ByteArraySerde,
-      CompressionType.NONE,
+      Compression.NONE,
       0L,
       mockTime.milliseconds(),
       false,
@@ -1060,7 +1061,7 @@ object KafkaMetadataLogTest {
     log.appendAsLeader(
       MemoryRecords.withRecords(
         log.endOffset().offset,
-        CompressionType.NONE,
+        Compression.NONE,
         epoch,
         (0 until numberOfRecords).map(number => new SimpleRecord(number.toString.getBytes)): _*
       ),
@@ -1071,7 +1072,7 @@ object KafkaMetadataLogTest {
   def append(snapshotWriter: RawSnapshotWriter, numberOfRecords: Int): Unit = {
     snapshotWriter.append(MemoryRecords.withRecords(
       0,
-      CompressionType.NONE,
+      Compression.NONE,
       0,
       (0 until numberOfRecords).map(number => new SimpleRecord(number.toString.getBytes)): _*
     ))
diff --git a/core/src/test/scala/kafka/server/LocalLeaderEndPointTest.scala b/core/src/test/scala/kafka/server/LocalLeaderEndPointTest.scala
index ac259ce07d9..9a5483ee5b6 100644
--- a/core/src/test/scala/kafka/server/LocalLeaderEndPointTest.scala
+++ b/core/src/test/scala/kafka/server/LocalLeaderEndPointTest.scala
@@ -21,13 +21,14 @@ import kafka.cluster.BrokerEndPoint
 import kafka.server.QuotaFactory.QuotaManagers
 import kafka.server.checkpoints.LazyOffsetCheckpoints
 import kafka.utils.{CoreUtils, Logging, TestUtils}
+import org.apache.kafka.common.compress.Compression
 import org.apache.kafka.common.{Node, TopicPartition, Uuid}
 import org.apache.kafka.common.message.LeaderAndIsrRequestData.LeaderAndIsrPartitionState
 import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderPartition
 import org.apache.kafka.common.message.OffsetForLeaderEpochResponseData.EpochEndOffset
 import org.apache.kafka.common.metrics.Metrics
 import org.apache.kafka.common.protocol.{ApiKeys, Errors}
-import org.apache.kafka.common.record.{CompressionType, MemoryRecords, SimpleRecord}
+import org.apache.kafka.common.record.{MemoryRecords, SimpleRecord}
 import org.apache.kafka.common.requests.LeaderAndIsrRequest
 import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
 import org.apache.kafka.server.common.OffsetAndEpoch
@@ -265,7 +266,7 @@ class LocalLeaderEndPointTest extends Logging {
   }
 
   private def records: MemoryRecords = {
-    MemoryRecords.withRecords(CompressionType.NONE,
+    MemoryRecords.withRecords(Compression.NONE,
       new SimpleRecord("first message".getBytes()),
       new SimpleRecord("second message".getBytes()),
       new SimpleRecord("third message".getBytes()),
diff --git a/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala b/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala
index 50dd667084c..b973bac55a6 100755
--- a/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala
+++ b/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala
@@ -26,6 +26,7 @@ import joptsimple._
 import kafka.log._
 import kafka.server.BrokerTopicStats
 import kafka.utils._
+import org.apache.kafka.common.compress.{Compression, GzipCompression, Lz4Compression, ZstdCompression}
 import org.apache.kafka.common.config.TopicConfig
 import org.apache.kafka.common.record._
 import org.apache.kafka.common.utils.{Time, Utils}
@@ -34,7 +35,7 @@ import org.apache.kafka.server.util.{KafkaScheduler, Scheduler}
 import org.apache.kafka.server.util.CommandLineUtils
 import org.apache.kafka.storage.internals.log.{LogConfig, LogDirFailureChannel, ProducerStateManagerConfig}
 
-import scala.math._
+import scala.math.max
 
 /**
  * This test does linear writes using either a kafka log or a file and measures throughput and latency.
@@ -66,29 +67,34 @@ object TestLinearWriteSpeed {
                            .describedAs("num_files")
                            .ofType(classOf[java.lang.Integer])
                            .defaultsTo(1)
-   val reportingIntervalOpt = parser.accepts("reporting-interval", "The number of ms between updates.")
+    val reportingIntervalOpt = parser.accepts("reporting-interval", "The number of ms between updates.")
                            .withRequiredArg
                            .describedAs("ms")
                            .ofType(classOf[java.lang.Long])
                            .defaultsTo(1000L)
-   val maxThroughputOpt = parser.accepts("max-throughput-mb", "The maximum throughput.")
+    val maxThroughputOpt = parser.accepts("max-throughput-mb", "The maximum throughput.")
                            .withRequiredArg
                            .describedAs("mb")
                            .ofType(classOf[java.lang.Integer])
                            .defaultsTo(Integer.MAX_VALUE)
-   val flushIntervalOpt = parser.accepts("flush-interval", "The number of messages between flushes")
+    val flushIntervalOpt = parser.accepts("flush-interval", "The number of messages between flushes")
                            .withRequiredArg()
                            .describedAs("message_count")
                            .ofType(classOf[java.lang.Long])
                            .defaultsTo(Long.MaxValue)
-   val compressionCodecOpt = parser.accepts("compression", "The compression codec to use")
+    val compressionCodecOpt = parser.accepts("compression", "The compression codec to use")
                             .withRequiredArg
                             .describedAs("codec")
                             .ofType(classOf[java.lang.String])
                             .defaultsTo(CompressionType.NONE.name)
-   val mmapOpt = parser.accepts("mmap", "Do writes to memory-mapped files.")
-   val channelOpt = parser.accepts("channel", "Do writes to file channels.")
-   val logOpt = parser.accepts("log", "Do writes to kafka logs.")
+    val compressionLevelOpt = parser.accepts("level", "The compression level to use")
+                            .withRequiredArg
+                            .describedAs("level")
+                            .ofType(classOf[java.lang.Integer])
+                            .defaultsTo(0)
+    val mmapOpt = parser.accepts("mmap", "Do writes to memory-mapped files.")
+    val channelOpt = parser.accepts("channel", "Do writes to file channels.")
+    val logOpt = parser.accepts("log", "Do writes to kafka logs.")
 
     val options = parser.parse(args : _*)
 
@@ -104,13 +110,22 @@ object TestLinearWriteSpeed {
     val messageSize = options.valueOf(messageSizeOpt).intValue
     val flushInterval = options.valueOf(flushIntervalOpt).longValue
     val compressionType = CompressionType.forName(options.valueOf(compressionCodecOpt))
+    val compressionBuilder = Compression.of(compressionType)
+    val compressionLevel = options.valueOf(compressionLevelOpt)
+    compressionType match {
+      case CompressionType.GZIP => compressionBuilder.asInstanceOf[GzipCompression.Builder].level(compressionLevel)
+      case CompressionType.LZ4 => compressionBuilder.asInstanceOf[Lz4Compression.Builder].level(compressionLevel)
+      case CompressionType.ZSTD => compressionBuilder.asInstanceOf[ZstdCompression.Builder].level(compressionLevel)
+      case _ => //Noop
+    }
+    val compression = compressionBuilder.build()
     val rand = new Random
     rand.nextBytes(buffer.array)
     val numMessages = bufferSize / (messageSize + Records.LOG_OVERHEAD)
     val createTime = System.currentTimeMillis
     val messageSet = {
       val records = (0 until numMessages).map(_ => new SimpleRecord(createTime, null, new Array[Byte](messageSize)))
-      MemoryRecords.withRecords(compressionType, records: _*)
+      MemoryRecords.withRecords(compression, records: _*)
     }
 
     val writables = new Array[Writable](numFiles)
@@ -197,7 +212,7 @@ object TestLinearWriteSpeed {
 
   class ChannelWritable(val file: File, val content: ByteBuffer) extends Writable {
     file.deleteOnExit()
-    val channel = FileChannel.open(file.toPath, StandardOpenOption.CREATE, StandardOpenOption.READ,
+    val channel: FileChannel = FileChannel.open(file.toPath, StandardOpenOption.CREATE, StandardOpenOption.READ,
       StandardOpenOption.WRITE)
     def write(): Int = {
       channel.write(content)
@@ -212,7 +227,7 @@ object TestLinearWriteSpeed {
 
   class LogWritable(val dir: File, config: LogConfig, scheduler: Scheduler, val messages: MemoryRecords) extends Writable {
     Utils.delete(dir)
-    val log = UnifiedLog(
+    val log: UnifiedLog = UnifiedLog(
       dir = dir,
       config = config,
       logStartOffset = 0L,
diff --git a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala
index 45ae991744a..2134dcfaaa0 100644
--- a/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala
+++ b/core/src/test/scala/unit/kafka/cluster/PartitionTest.scala
@@ -47,6 +47,7 @@ import java.util.Optional
 import java.util.concurrent.{CountDownLatch, Semaphore}
 import kafka.server.metadata.{KRaftMetadataCache, ZkMetadataCache}
 import org.apache.kafka.clients.ClientResponse
+import org.apache.kafka.common.compress.Compression
 import org.apache.kafka.common.config.TopicConfig
 import org.apache.kafka.common.network.ListenerName
 import org.apache.kafka.common.replica.ClientMetadata
@@ -159,7 +160,7 @@ class PartitionTest extends AbstractPartitionTest {
       val recordArray = (1 to count).map { i =>
         new SimpleRecord(s"$i".getBytes)
       }
-      val records = MemoryRecords.withRecords(0L, CompressionType.NONE, leaderEpoch,
+      val records = MemoryRecords.withRecords(0L, Compression.NONE, leaderEpoch,
         recordArray: _*)
       log.appendAsLeader(records, leaderEpoch = leaderEpoch)
     }
@@ -239,11 +240,11 @@ class PartitionTest extends AbstractPartitionTest {
     val leaderEpoch = 8
 
     val log = logManager.getOrCreateLog(topicPartition, topicId = None)
-    log.appendAsLeader(MemoryRecords.withRecords(0L, CompressionType.NONE, 0,
+    log.appendAsLeader(MemoryRecords.withRecords(0L, Compression.NONE, 0,
       new SimpleRecord("k1".getBytes, "v1".getBytes),
       new SimpleRecord("k2".getBytes, "v2".getBytes)
     ), leaderEpoch = 0)
-    log.appendAsLeader(MemoryRecords.withRecords(0L, CompressionType.NONE, 5,
+    log.appendAsLeader(MemoryRecords.withRecords(0L, Compression.NONE, 5,
       new SimpleRecord("k3".getBytes, "v3".getBytes),
       new SimpleRecord("k4".getBytes, "v4".getBytes)
     ), leaderEpoch = 5)
@@ -505,7 +506,7 @@ class PartitionTest extends AbstractPartitionTest {
 
     // Write records with duplicate keys to current replica and roll at offset 6
     val currentLog = partition.log.get
-    currentLog.appendAsLeader(MemoryRecords.withRecords(0L, CompressionType.NONE, 0,
+    currentLog.appendAsLeader(MemoryRecords.withRecords(0L, Compression.NONE, 0,
       new SimpleRecord("k1".getBytes, "v1".getBytes),
       new SimpleRecord("k1".getBytes, "v2".getBytes),
       new SimpleRecord("k1".getBytes, "v3".getBytes),
@@ -514,7 +515,7 @@ class PartitionTest extends AbstractPartitionTest {
       new SimpleRecord("k2".getBytes, "v6".getBytes)
     ), leaderEpoch = 0)
     currentLog.roll()
-    currentLog.appendAsLeader(MemoryRecords.withRecords(0L, CompressionType.NONE, 0,
+    currentLog.appendAsLeader(MemoryRecords.withRecords(0L, Compression.NONE, 0,
       new SimpleRecord("k3".getBytes, "v7".getBytes),
       new SimpleRecord("k4".getBytes, "v8".getBytes)
     ), leaderEpoch = 0)
@@ -522,7 +523,7 @@ class PartitionTest extends AbstractPartitionTest {
     // Write to the future replica as if the log had been compacted, and do not roll the segment
 
     val buffer = ByteBuffer.allocate(1024)
-    val builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, CompressionType.NONE,
+    val builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, Compression.NONE,
       TimestampType.CREATE_TIME, 0L, RecordBatch.NO_TIMESTAMP, 0)
     builder.appendWithOffset(2L, new SimpleRecord("k1".getBytes, "v3".getBytes))
     builder.appendWithOffset(5L, new SimpleRecord("k2".getBytes, "v6".getBytes))
@@ -1184,7 +1185,7 @@ class PartitionTest extends AbstractPartitionTest {
   def createRecords(records: Iterable[SimpleRecord], baseOffset: Long, partitionLeaderEpoch: Int = 0): MemoryRecords = {
     val buf = ByteBuffer.allocate(DefaultRecordBatch.sizeInBytes(records.asJava))
     val builder = MemoryRecords.builder(
-      buf, RecordBatch.CURRENT_MAGIC_VALUE, CompressionType.NONE, TimestampType.LOG_APPEND_TIME,
+      buf, RecordBatch.CURRENT_MAGIC_VALUE, Compression.NONE, TimestampType.LOG_APPEND_TIME,
       baseOffset, time.milliseconds, partitionLeaderEpoch)
     records.foreach(builder.append)
     builder.build()
@@ -1197,7 +1198,7 @@ class PartitionTest extends AbstractPartitionTest {
     val producerEpoch = 0.toShort
     val isTransactional = false
     val buf = ByteBuffer.allocate(DefaultRecordBatch.sizeInBytes(records.asJava))
-    val builder = MemoryRecords.builder(buf, CompressionType.NONE, baseOffset, producerId,
+    val builder = MemoryRecords.builder(buf, Compression.NONE, baseOffset, producerId,
       producerEpoch, baseSequence, isTransactional)
     records.foreach(builder.append)
     builder.build()
@@ -1210,7 +1211,7 @@ class PartitionTest extends AbstractPartitionTest {
     val producerEpoch = 0.toShort
     val isTransactional = true
     val buf = ByteBuffer.allocate(DefaultRecordBatch.sizeInBytes(records.asJava))
-    val builder = MemoryRecords.builder(buf, CompressionType.NONE, baseOffset, producerId,
+    val builder = MemoryRecords.builder(buf, Compression.NONE, baseOffset, producerId,
       producerEpoch, baseSequence, isTransactional)
     records.foreach(builder.append)
     builder.build()
@@ -3180,7 +3181,7 @@ class PartitionTest extends AbstractPartitionTest {
     assertEquals(Optional.of(new EpochEntry(leaderEpoch, 0L)), leaderLog.leaderEpochCache.asJava.flatMap(_.latestEntry))
 
     // Write to the log to increment the log end offset.
-    leaderLog.appendAsLeader(MemoryRecords.withRecords(0L, CompressionType.NONE, 0,
+    leaderLog.appendAsLeader(MemoryRecords.withRecords(0L, Compression.NONE, 0,
       new SimpleRecord("k1".getBytes, "v1".getBytes),
       new SimpleRecord("k1".getBytes, "v1".getBytes)
     ), leaderEpoch = leaderEpoch)
@@ -3671,7 +3672,7 @@ class PartitionTest extends AbstractPartitionTest {
 
   private def seedLogData(log: UnifiedLog, numRecords: Int, leaderEpoch: Int): Unit = {
     for (i <- 0 until numRecords) {
-      val records = MemoryRecords.withRecords(0L, CompressionType.NONE, leaderEpoch,
+      val records = MemoryRecords.withRecords(0L, Compression.NONE, leaderEpoch,
         new SimpleRecord(s"k$i".getBytes, s"v$i".getBytes))
       log.appendAsLeader(records, leaderEpoch)
     }
diff --git a/core/src/test/scala/unit/kafka/coordinator/group/CoordinatorLoaderImplTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/CoordinatorLoaderImplTest.scala
index 3d3eeca2582..767549a7b53 100644
--- a/core/src/test/scala/unit/kafka/coordinator/group/CoordinatorLoaderImplTest.scala
+++ b/core/src/test/scala/unit/kafka/coordinator/group/CoordinatorLoaderImplTest.scala
@@ -19,8 +19,9 @@ package kafka.coordinator.group
 import kafka.log.UnifiedLog
 import kafka.server.ReplicaManager
 import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.compress.Compression
 import org.apache.kafka.common.errors.NotLeaderOrFollowerException
-import org.apache.kafka.common.record.{CompressionType, ControlRecordType, EndTransactionMarker, FileRecords, MemoryRecords, RecordBatch, SimpleRecord}
+import org.apache.kafka.common.record.{ControlRecordType, EndTransactionMarker, FileRecords, MemoryRecords, RecordBatch, SimpleRecord}
 import org.apache.kafka.common.requests.TransactionResult
 import org.apache.kafka.common.utils.{MockTime, Time}
 import org.apache.kafka.coordinator.group.runtime.CoordinatorLoader.UnknownRecordTypeException
@@ -639,13 +640,13 @@ class CoordinatorLoaderImplTest {
     val memoryRecords = if (producerId == RecordBatch.NO_PRODUCER_ID) {
       MemoryRecords.withRecords(
         startOffset,
-        CompressionType.NONE,
+        Compression.NONE,
         records: _*
       )
     } else {
       MemoryRecords.withTransactionalRecords(
         startOffset,
-        CompressionType.NONE,
+        Compression.NONE,
         producerId,
         producerEpoch,
         0,
diff --git a/core/src/test/scala/unit/kafka/coordinator/group/CoordinatorPartitionWriterTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/CoordinatorPartitionWriterTest.scala
index c833f6a3b07..cdb41bb9d0e 100644
--- a/core/src/test/scala/unit/kafka/coordinator/group/CoordinatorPartitionWriterTest.scala
+++ b/core/src/test/scala/unit/kafka/coordinator/group/CoordinatorPartitionWriterTest.scala
@@ -18,9 +18,10 @@ package kafka.coordinator.group
 
 import kafka.server.ReplicaManager
 import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.compress.Compression
 import org.apache.kafka.common.errors.NotLeaderOrFollowerException
 import org.apache.kafka.common.protocol.{ApiKeys, Errors}
-import org.apache.kafka.common.record.{CompressionType, MemoryRecords, RecordBatch, SimpleRecord}
+import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, SimpleRecord}
 import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
 import org.apache.kafka.coordinator.group.runtime.PartitionWriter
 import org.apache.kafka.storage.internals.log.{AppendOrigin, LogConfig, VerificationGuard}
@@ -122,7 +123,7 @@ class CoordinatorPartitionWriterTest {
     })
 
     val batch = MemoryRecords.withRecords(
-      CompressionType.NONE,
+      Compression.NONE,
       new SimpleRecord(
         0L,
         "foo".getBytes(Charset.defaultCharset()),
@@ -223,7 +224,7 @@ class CoordinatorPartitionWriterTest {
     })
 
     val batch = MemoryRecords.withRecords(
-      CompressionType.NONE,
+      Compression.NONE,
       new SimpleRecord(
         0L,
         "foo".getBytes(Charset.defaultCharset()),
diff --git a/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala
index 94d9697f184..16459cbed2f 100644
--- a/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/coordinator/group/GroupMetadataManagerTest.scala
@@ -32,6 +32,7 @@ import kafka.utils.TestUtils
 import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor
 import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.Subscription
 import org.apache.kafka.clients.consumer.internals.ConsumerProtocol
+import org.apache.kafka.common.compress.Compression
 import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid}
 import org.apache.kafka.common.internals.Topic
 import org.apache.kafka.common.metrics.{JmxReporter, KafkaMetricsContext, Metrics => kMetrics}
@@ -148,7 +149,7 @@ class GroupMetadataManagerTest {
     )
 
     val offsetCommitRecords = createCommittedOffsetRecords(committedOffsets)
-    val records = MemoryRecords.withRecords(startOffset, CompressionType.NONE, offsetCommitRecords.toArray: _*)
+    val records = MemoryRecords.withRecords(startOffset, Compression.NONE, offsetCommitRecords.toArray: _*)
     expectGroupMetadataLoad(groupMetadataTopicPartition, startOffset, records)
 
     groupMetadataManager.loadGroupsAndOffsets(groupMetadataTopicPartition, groupEpoch, _ => (), 0L)
@@ -177,7 +178,7 @@ class GroupMetadataManagerTest {
 
     val offsetCommitRecords = createCommittedOffsetRecords(committedOffsets)
     val groupMetadataRecord = buildEmptyGroupRecord(generation, protocolType)
-    val records = MemoryRecords.withRecords(startOffset, CompressionType.NONE,
+    val records = MemoryRecords.withRecords(startOffset, Compression.NONE,
       (offsetCommitRecords ++ Seq(groupMetadataRecord)).toArray: _*)
 
     expectGroupMetadataLoad(groupMetadataTopicPartition, startOffset, records)
@@ -566,7 +567,7 @@ class GroupMetadataManagerTest {
   }
 
   private def appendConsumerOffsetCommit(buffer: ByteBuffer, baseOffset: Long, offsets: Map[TopicPartition, Long]) = {
-    val builder = MemoryRecords.builder(buffer, CompressionType.NONE, TimestampType.LOG_APPEND_TIME, baseOffset)
+    val builder = MemoryRecords.builder(buffer, Compression.NONE, TimestampType.LOG_APPEND_TIME, baseOffset)
     val commitRecords = createCommittedOffsetRecords(offsets)
     commitRecords.foreach(builder.append)
     builder.build()
@@ -575,7 +576,7 @@ class GroupMetadataManagerTest {
 
   private def appendTransactionalOffsetCommits(buffer: ByteBuffer, producerId: Long, producerEpoch: Short,
                                                baseOffset: Long, offsets: Map[TopicPartition, Long]): Int = {
-    val builder = MemoryRecords.builder(buffer, CompressionType.NONE, baseOffset, producerId, producerEpoch, 0, true)
+    val builder = MemoryRecords.builder(buffer, Compression.NONE, baseOffset, producerId, producerEpoch, 0, true)
     val commitRecords = createCommittedOffsetRecords(offsets)
     commitRecords.foreach(builder.append)
     builder.build()
@@ -584,7 +585,7 @@ class GroupMetadataManagerTest {
 
   private def completeTransactionalOffsetCommit(buffer: ByteBuffer, producerId: Long, producerEpoch: Short, baseOffset: Long,
                                                 isCommit: Boolean): Int = {
-    val builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, CompressionType.NONE,
+    val builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, Compression.NONE,
       TimestampType.LOG_APPEND_TIME, baseOffset, time.milliseconds(), producerId, producerEpoch, 0, true, true,
       RecordBatch.NO_PARTITION_LEADER_EPOCH)
     val controlRecordType = if (isCommit) ControlRecordType.COMMIT else ControlRecordType.ABORT
@@ -608,7 +609,7 @@ class GroupMetadataManagerTest {
 
     val offsetCommitRecords = createCommittedOffsetRecords(committedOffsets)
     val tombstone = new SimpleRecord(GroupMetadataManager.offsetCommitKey(groupId, tombstonePartition), null)
-    val records = MemoryRecords.withRecords(startOffset, CompressionType.NONE,
+    val records = MemoryRecords.withRecords(startOffset, Compression.NONE,
       (offsetCommitRecords ++ Seq(tombstone)).toArray: _*)
 
     expectGroupMetadataLoad(groupMetadataTopicPartition, startOffset, records)
@@ -647,7 +648,7 @@ class GroupMetadataManagerTest {
     val memberId = "98098230493"
     val groupMetadataRecord = buildStableGroupRecordWithMember(generation, protocolType, protocol, memberId)
 
-    val records = MemoryRecords.withRecords(startOffset, CompressionType.NONE,
+    val records = MemoryRecords.withRecords(startOffset, Compression.NONE,
       (offsetCommitRecords ++ Seq(groupMetadataRecord)).toArray: _*)
 
     expectGroupMetadataLoad(groupMetadataTopicPartition, startOffset, records)
@@ -756,7 +757,7 @@ class GroupMetadataManagerTest {
     val groupMetadataRecord = buildStableGroupRecordWithMember(generation = 15,
       protocolType = "consumer", protocol = "range", memberId)
     val groupMetadataTombstone = new SimpleRecord(GroupMetadataManager.groupMetadataKey(groupId), null)
-    val records = MemoryRecords.withRecords(startOffset, CompressionType.NONE,
+    val records = MemoryRecords.withRecords(startOffset, Compression.NONE,
       Seq(groupMetadataRecord, groupMetadataTombstone).toArray: _*)
 
     expectGroupMetadataLoad(groupMetadataTopicPartition, startOffset, records)
@@ -784,7 +785,7 @@ class GroupMetadataManagerTest {
     val offsetCommitRecords = createCommittedOffsetRecords(committedOffsets)
     val groupMetadataRecord = buildStableGroupRecordWithMember(generation = 15,
       protocolType = "consumer", protocol = "range", memberId, new Array[Byte](assignmentSize))
-    val records = MemoryRecords.withRecords(startOffset, CompressionType.NONE,
+    val records = MemoryRecords.withRecords(startOffset, Compression.NONE,
       (offsetCommitRecords ++ Seq(groupMetadataRecord)).toArray: _*)
 
     expectGroupMetadataLoad(groupMetadataTopicPartition, startOffset, records)
@@ -845,7 +846,7 @@ class GroupMetadataManagerTest {
     val memberId = "98098230493"
     val groupMetadataRecord = buildStableGroupRecordWithMember(generation, protocolType, protocol, memberId)
     val groupMetadataTombstone = new SimpleRecord(GroupMetadataManager.groupMetadataKey(groupId), null)
-    val records = MemoryRecords.withRecords(startOffset, CompressionType.NONE,
+    val records = MemoryRecords.withRecords(startOffset, Compression.NONE,
       (Seq(groupMetadataRecord, groupMetadataTombstone) ++ offsetCommitRecords).toArray: _*)
 
     expectGroupMetadataLoad(groupMetadataTopicPartition, startOffset, records)
@@ -879,14 +880,14 @@ class GroupMetadataManagerTest {
 
     val segment1MemberId = "a"
     val segment1Offsets = Map(tp0 -> 23L, tp1 -> 455L, tp3 -> 42L)
-    val segment1Records = MemoryRecords.withRecords(startOffset, CompressionType.NONE,
+    val segment1Records = MemoryRecords.withRecords(startOffset, Compression.NONE,
       (createCommittedOffsetRecords(segment1Offsets) ++ Seq(buildStableGroupRecordWithMember(
         generation, protocolType, protocol, segment1MemberId))).toArray: _*)
     val segment1End = startOffset + segment1Records.records.asScala.size
 
     val segment2MemberId = "b"
     val segment2Offsets = Map(tp0 -> 33L, tp2 -> 8992L, tp3 -> 10L)
-    val segment2Records = MemoryRecords.withRecords(segment1End, CompressionType.NONE,
+    val segment2Records = MemoryRecords.withRecords(segment1End, Compression.NONE,
       (createCommittedOffsetRecords(segment2Offsets) ++ Seq(buildStableGroupRecordWithMember(
         generation, protocolType, protocol, segment2MemberId))).toArray: _*)
     val segment2End = segment1End + segment2Records.records.asScala.size
@@ -2352,7 +2353,7 @@ class GroupMetadataManagerTest {
     val offsetCommitRecords = createCommittedOffsetRecords(committedOffsets, metadataVersion = metadataVersion, retentionTimeOpt = Some(100))
     val memberId = "98098230493"
     val groupMetadataRecord = buildStableGroupRecordWithMember(generation, protocolType, protocol, memberId, metadataVersion = metadataVersion)
-    val records = MemoryRecords.withRecords(startOffset, CompressionType.NONE,
+    val records = MemoryRecords.withRecords(startOffset, Compression.NONE,
       (offsetCommitRecords ++ Seq(groupMetadataRecord)).toArray: _*)
 
     expectGroupMetadataLoad(groupMetadataTopicPartition, startOffset, records)
@@ -2391,7 +2392,7 @@ class GroupMetadataManagerTest {
     val offsetCommitRecords = createCommittedOffsetRecords(committedOffsets, retentionTimeOpt = Some(100))
     val memberId = "98098230493"
     val groupMetadataRecord = buildStableGroupRecordWithMember(generation, protocolType, protocol, memberId)
-    val records = MemoryRecords.withRecords(startOffset, CompressionType.NONE,
+    val records = MemoryRecords.withRecords(startOffset, Compression.NONE,
       (offsetCommitRecords ++ Seq(groupMetadataRecord)).toArray: _*)
 
     expectGroupMetadataLoad(groupMetadataTopicPartition, startOffset, records)
@@ -2731,7 +2732,7 @@ class GroupMetadataManagerTest {
 
     val offsetCommitRecords = createCommittedOffsetRecords(committedOffsets)
     val groupMetadataRecord = buildEmptyGroupRecord(generation, protocolType)
-    val records = MemoryRecords.withRecords(startOffset, CompressionType.NONE,
+    val records = MemoryRecords.withRecords(startOffset, Compression.NONE,
       (offsetCommitRecords ++ Seq(groupMetadataRecord)).toArray: _*)
 
     // Prepend empty control batch to valid records
@@ -2948,7 +2949,7 @@ class GroupMetadataManagerTest {
     val offsetCommitRecords = createCommittedOffsetRecords(committedOffsets)
     val groupMetadataRecord = buildStableGroupRecordWithMember(generation = 15,
       protocolType = "consumer", protocol = "range", memberId)
-    val records = MemoryRecords.withRecords(startOffset, CompressionType.NONE,
+    val records = MemoryRecords.withRecords(startOffset, Compression.NONE,
       (offsetCommitRecords ++ Seq(groupMetadataRecord)).toArray: _*)
 
     expectGroupMetadataLoad(groupMetadataTopicPartition, startOffset, records)
@@ -2995,7 +2996,7 @@ class GroupMetadataManagerTest {
     val unknownRecord1 = new SimpleRecord(unknownMessage1, unknownMessage1)
     val unknownRecord2 = new SimpleRecord(unknownMessage2, unknownMessage2)
 
-    val records = MemoryRecords.withRecords(startOffset, CompressionType.NONE,
+    val records = MemoryRecords.withRecords(startOffset, Compression.NONE,
       (offsetCommitRecords ++ Seq(unknownRecord1, unknownRecord2) ++ Seq(groupMetadataRecord)).toArray: _*)
 
     expectGroupMetadataLoad(groupTopicPartition, startOffset, records)
diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala
index d57a8e974c6..2dcd6fcc7ab 100644
--- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala
+++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionCoordinatorConcurrencyTest.scala
@@ -26,11 +26,12 @@ import kafka.log.UnifiedLog
 import kafka.server.{KafkaConfig, MetadataCache, RequestLocal}
 import kafka.utils.{Pool, TestUtils}
 import org.apache.kafka.clients.{ClientResponse, NetworkClient}
+import org.apache.kafka.common.compress.Compression
 import org.apache.kafka.common.internals.Topic.TRANSACTION_STATE_TOPIC_NAME
 import org.apache.kafka.common.metrics.Metrics
 import org.apache.kafka.common.network.ListenerName
 import org.apache.kafka.common.protocol.{ApiKeys, Errors}
-import org.apache.kafka.common.record.{CompressionType, FileRecords, MemoryRecords, RecordBatch, SimpleRecord}
+import org.apache.kafka.common.record.{FileRecords, MemoryRecords, RecordBatch, SimpleRecord}
 import org.apache.kafka.common.requests._
 import org.apache.kafka.common.utils.{LogContext, MockTime, ProducerIdAndEpoch}
 import org.apache.kafka.common.{Node, TopicPartition}
@@ -466,7 +467,7 @@ class TransactionCoordinatorConcurrencyTest extends AbstractCoordinatorConcurren
 
     val topicPartition = new TopicPartition(TRANSACTION_STATE_TOPIC_NAME, partitionId)
     val startOffset = replicaManager.getLogEndOffset(topicPartition).getOrElse(20L)
-    val records = MemoryRecords.withRecords(startOffset, CompressionType.NONE, txnRecordsByPartition(partitionId).toArray: _*)
+    val records = MemoryRecords.withRecords(startOffset, Compression.NONE, txnRecordsByPartition(partitionId).toArray: _*)
     val endOffset = startOffset + records.records.asScala.size
 
     when(logMock.logStartOffset).thenReturn(startOffset)
diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionLogTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionLogTest.scala
index 12dc8849dd1..8f7a92ed2f1 100644
--- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionLogTest.scala
+++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionLogTest.scala
@@ -21,10 +21,11 @@ import kafka.internals.generated.TransactionLogKey
 import kafka.internals.generated.TransactionLogValue
 import kafka.utils.TestUtils
 import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.compress.Compression
 import org.apache.kafka.common.protocol.{ByteBufferAccessor, MessageUtil}
 import org.apache.kafka.common.protocol.types.Field.TaggedFieldsSection
 import org.apache.kafka.common.protocol.types.{CompactArrayOf, Field, Schema, Struct, Type}
-import org.apache.kafka.common.record.{CompressionType, MemoryRecords, SimpleRecord}
+import org.apache.kafka.common.record.{MemoryRecords, SimpleRecord}
 import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows, assertTrue}
 import org.junit.jupiter.api.Test
 
@@ -84,7 +85,7 @@ class TransactionLogTest {
       new SimpleRecord(keyBytes, valueBytes)
     }.toSeq
 
-    val records = MemoryRecords.withRecords(0, CompressionType.NONE, txnRecords: _*)
+    val records = MemoryRecords.withRecords(0, Compression.NONE, txnRecords: _*)
 
     var count = 0
     for (record <- records.records.asScala) {
diff --git a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala
index 907e4c9c17e..adc1dcf644f 100644
--- a/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/coordinator/transaction/TransactionStateManagerTest.scala
@@ -28,6 +28,7 @@ import kafka.server.{ReplicaManager, RequestLocal}
 import kafka.utils.{Pool, TestUtils}
 import kafka.zk.KafkaZkClient
 import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.compress.Compression
 import org.apache.kafka.common.internals.Topic.TRANSACTION_STATE_TOPIC_NAME
 import org.apache.kafka.common.metrics.{JmxReporter, KafkaMetricsContext, Metrics}
 import org.apache.kafka.common.protocol.{Errors, MessageUtil}
@@ -166,7 +167,7 @@ class TransactionStateManagerTest {
     txnMetadata1.addPartitions(Set[TopicPartition](
       new TopicPartition("topic1", 0),
       new TopicPartition("topic1", 1)))
-    val records = MemoryRecords.withRecords(startOffset, CompressionType.NONE,
+    val records = MemoryRecords.withRecords(startOffset, Compression.NONE,
       new SimpleRecord(txnMessageKeyBytes1, TransactionLog.valueToBytes(txnMetadata1.prepareNoTransit())))
 
     // We create a latch which is awaited while the log is loading. This ensures that the deletion
@@ -250,7 +251,7 @@ class TransactionStateManagerTest {
     txnRecords += new SimpleRecord(txnMessageKeyBytes2, TransactionLog.valueToBytes(txnMetadata2.prepareNoTransit()))
 
     val startOffset = 15L   // it should work for any start offset
-    val records = MemoryRecords.withRecords(startOffset, CompressionType.NONE, txnRecords.toArray: _*)
+    val records = MemoryRecords.withRecords(startOffset, Compression.NONE, txnRecords.toArray: _*)
 
     prepareTxnLog(topicPartition, startOffset, records)
 
@@ -878,7 +879,7 @@ class TransactionStateManagerTest {
 
     txnRecords += new SimpleRecord(txnMessageKeyBytes1, TransactionLog.valueToBytes(txnMetadata1.prepareNoTransit()))
     val startOffset = 0L
-    val records = MemoryRecords.withRecords(startOffset, CompressionType.NONE, txnRecords.toArray: _*)
+    val records = MemoryRecords.withRecords(startOffset, Compression.NONE, txnRecords.toArray: _*)
 
     prepareTxnLog(topicPartition, 0, records)
 
@@ -1026,7 +1027,7 @@ class TransactionStateManagerTest {
       val partitionId = transactionManager.partitionFor(transactionalId1)
       val topicPartition = new TopicPartition(TRANSACTION_STATE_TOPIC_NAME, partitionId)
       val expectedTombstone = new SimpleRecord(time.milliseconds(), TransactionLog.keyToBytes(transactionalId1), null)
-      val expectedRecords = MemoryRecords.withRecords(TransactionLog.EnforcedCompressionType, expectedTombstone)
+      val expectedRecords = MemoryRecords.withRecords(TransactionLog.EnforcedCompression, expectedTombstone)
       assertEquals(Set(topicPartition), appendedRecords.keySet)
       assertEquals(Seq(expectedRecords), appendedRecords(topicPartition).toSeq)
     } else {
@@ -1041,7 +1042,7 @@ class TransactionStateManagerTest {
 
     txnRecords += new SimpleRecord(txnMessageKeyBytes1, TransactionLog.valueToBytes(txnMetadata1.prepareNoTransit()))
     val startOffset = 0L
-    val records = MemoryRecords.withRecords(startOffset, CompressionType.NONE, txnRecords.toArray: _*)
+    val records = MemoryRecords.withRecords(startOffset, Compression.NONE, txnRecords.toArray: _*)
 
     prepareTxnLog(topicPartition, 0, records)
 
@@ -1148,7 +1149,7 @@ class TransactionStateManagerTest {
     txnRecords += new SimpleRecord(txnMessageKeyBytes1, TransactionLog.valueToBytes(txnMetadata1.prepareNoTransit()))
 
     val startOffset = 15L
-    val records = MemoryRecords.withRecords(startOffset, CompressionType.NONE, txnRecords.toArray: _*)
+    val records = MemoryRecords.withRecords(startOffset, Compression.NONE, txnRecords.toArray: _*)
 
     prepareTxnLog(topicPartition, startOffset, records)
     transactionManager.loadTransactionsForTxnTopicPartition(partitionId, 0, (_, _, _, _) => ())
@@ -1171,7 +1172,7 @@ class TransactionStateManagerTest {
     val unknownMessage = MessageUtil.toVersionPrefixedBytes(Short.MaxValue, unknownKey)
     val unknownRecord = new SimpleRecord(unknownMessage, unknownMessage)
 
-    val records = MemoryRecords.withRecords(startOffset, CompressionType.NONE,
+    val records = MemoryRecords.withRecords(startOffset, Compression.NONE,
       (Seq(unknownRecord) ++ txnRecords).toArray: _*)
 
     prepareTxnLog(topicPartition, 0, records)
diff --git a/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala
index 6db9b1fa2d1..ae35d4a58a8 100644
--- a/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala
+++ b/core/src/test/scala/unit/kafka/log/AbstractLogCleanerIntegrationTest.scala
@@ -23,8 +23,9 @@ import kafka.server.BrokerTopicStats
 import kafka.utils.{Pool, TestUtils}
 import kafka.utils.Implicits._
 import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.compress.Compression
 import org.apache.kafka.common.config.TopicConfig
-import org.apache.kafka.common.record.{CompressionType, MemoryRecords, RecordBatch}
+import org.apache.kafka.common.record.{MemoryRecords, RecordBatch}
 import org.apache.kafka.common.utils.Utils
 import org.apache.kafka.coordinator.transaction.TransactionLogConfigs
 import org.apache.kafka.server.util.MockTime
@@ -142,7 +143,7 @@ abstract class AbstractLogCleanerIntegrationTest {
   def counter: Int = ctr
   def incCounter(): Unit = ctr += 1
 
-  def writeDups(numKeys: Int, numDups: Int, log: UnifiedLog, codec: CompressionType,
+  def writeDups(numKeys: Int, numDups: Int, log: UnifiedLog, codec: Compression,
                 startKey: Int = 0, magicValue: Byte = RecordBatch.CURRENT_MAGIC_VALUE): Seq[(Int, String, Long)] = {
     for (_ <- 0 until numDups; key <- startKey until (startKey + numKeys)) yield {
       val value = counter.toString
@@ -155,7 +156,7 @@ abstract class AbstractLogCleanerIntegrationTest {
     }
   }
 
-  def createLargeSingleMessageSet(key: Int, messageFormatVersion: Byte, codec: CompressionType): (String, MemoryRecords) = {
+  def createLargeSingleMessageSet(key: Int, messageFormatVersion: Byte, codec: Compression): (String, MemoryRecords) = {
     def messageValue(length: Int): String = {
       val random = new Random(0)
       new String(random.alphanumeric.take(length).toArray)
diff --git a/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala b/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala
index b53c4c2ddba..07e70e97f0b 100755
--- a/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala
+++ b/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala
@@ -19,6 +19,7 @@ package kafka.log
 
 import kafka.server.BrokerTopicStats
 import kafka.utils._
+import org.apache.kafka.common.compress.Compression
 import org.apache.kafka.common.config.TopicConfig
 import org.apache.kafka.common.record.{CompressionType, MemoryRecords, RecordBatch, SimpleRecord}
 import org.apache.kafka.common.utils.Utils
@@ -50,10 +51,10 @@ class BrokerCompressionTest {
    */
   @ParameterizedTest
   @MethodSource(Array("parameters"))
-  def testBrokerSideCompression(messageCompression: String, brokerCompression: String): Unit = {
-    val messageCompressionType = CompressionType.forName(messageCompression)
+  def testBrokerSideCompression(messageCompressionType: CompressionType, brokerCompressionType: BrokerCompressionType): Unit = {
+    val messageCompression = Compression.of(messageCompressionType).build()
     val logProps = new Properties()
-    logProps.put(TopicConfig.COMPRESSION_TYPE_CONFIG, brokerCompression)
+    logProps.put(TopicConfig.COMPRESSION_TYPE_CONFIG, brokerCompressionType.name)
     /*configure broker-side compression  */
     val log = UnifiedLog(
       dir = logDir,
@@ -72,7 +73,7 @@ class BrokerCompressionTest {
     )
 
     /* append two messages */
-    log.appendAsLeader(MemoryRecords.withRecords(messageCompressionType, 0,
+    log.appendAsLeader(MemoryRecords.withRecords(messageCompression, 0,
           new SimpleRecord("hello".getBytes), new SimpleRecord("there".getBytes)), leaderEpoch = 0)
 
     def readBatch(offset: Int): RecordBatch = {
@@ -83,9 +84,9 @@ class BrokerCompressionTest {
       fetchInfo.records.batches.iterator.next()
     }
 
-    if (!brokerCompression.equals("producer")) {
-      val brokerCompressionType = BrokerCompressionType.forName(brokerCompression).targetCompressionType(null)
-      assertEquals(brokerCompressionType, readBatch(0).compressionType, "Compression at offset 0 should produce " + brokerCompressionType)
+    if (brokerCompressionType != BrokerCompressionType.PRODUCER) {
+      val targetCompression = BrokerCompressionType.targetCompression(log.config.compression, null)
+      assertEquals(targetCompression.`type`(), readBatch(0).compressionType, "Compression at offset 0 should produce " + brokerCompressionType)
     }
     else
       assertEquals(messageCompressionType, readBatch(0).compressionType, "Compression at offset 0 should produce " + messageCompressionType)
@@ -98,7 +99,7 @@ object BrokerCompressionTest {
     java.util.Arrays.stream(
       for (brokerCompression <- BrokerCompressionType.values;
            messageCompression <- CompressionType.values
-      ) yield Arguments.of(messageCompression.name, brokerCompression.name)
+      ) yield Arguments.of(messageCompression, brokerCompression)
     )
   }
 }
diff --git a/core/src/test/scala/unit/kafka/log/LocalLogTest.scala b/core/src/test/scala/unit/kafka/log/LocalLogTest.scala
index 32513a89158..a0ba0c478c7 100644
--- a/core/src/test/scala/unit/kafka/log/LocalLogTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LocalLogTest.scala
@@ -24,9 +24,10 @@ import java.util.regex.Pattern
 import java.util.Collections
 import kafka.server.KafkaConfig
 import kafka.utils.TestUtils
+import org.apache.kafka.common.compress.Compression
 import org.apache.kafka.common.{KafkaException, TopicPartition}
 import org.apache.kafka.common.errors.KafkaStorageException
-import org.apache.kafka.common.record.{CompressionType, MemoryRecords, Record, SimpleRecord}
+import org.apache.kafka.common.record.{MemoryRecords, Record, SimpleRecord}
 import org.apache.kafka.common.utils.{Time, Utils}
 import org.apache.kafka.server.util.{MockTime, Scheduler}
 import org.apache.kafka.storage.internals.log.{FetchDataInfo, LogConfig, LogDirFailureChannel, LogFileUtils, LogOffsetMetadata, LogSegment, LogSegments}
@@ -99,7 +100,7 @@ class LocalLogTest {
     log.append(lastOffset = initialOffset + records.size - 1,
       largestTimestamp = records.head.timestamp,
       shallowOffsetOfMaxTimestamp = initialOffset,
-      records = MemoryRecords.withRecords(initialOffset, CompressionType.NONE, 0, records.toList : _*))
+      records = MemoryRecords.withRecords(initialOffset, Compression.NONE, 0, records.toList : _*))
   }
 
   private def readRecords(log: LocalLog = log,
diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala
index b85014ca50a..cecf2c326b6 100644
--- a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala
@@ -21,7 +21,8 @@ import java.io.PrintWriter
 import com.yammer.metrics.core.{Gauge, MetricName}
 import kafka.utils.TestUtils
 import org.apache.kafka.common.TopicPartition
-import org.apache.kafka.common.record.{CompressionType, RecordBatch}
+import org.apache.kafka.common.compress.Compression
+import org.apache.kafka.common.record.RecordBatch
 import org.apache.kafka.server.metrics.KafkaYammerMetrics
 import org.apache.kafka.server.util.MockTime
 import org.junit.jupiter.api.Assertions._
@@ -35,7 +36,7 @@ import scala.jdk.CollectionConverters._
   */
 class LogCleanerIntegrationTest extends AbstractLogCleanerIntegrationTest {
 
-  val codec: CompressionType = CompressionType.LZ4
+  val codec: Compression = Compression.lz4().build()
 
   val time = new MockTime()
   val topicPartitions = Array(new TopicPartition("log", 0), new TopicPartition("log", 1), new TopicPartition("log", 2))
@@ -141,7 +142,7 @@ class LogCleanerIntegrationTest extends AbstractLogCleanerIntegrationTest {
     val log = cleaner.logs.get(topicPartitions(0))
 
     val T0 = time.milliseconds
-    writeKeyDups(numKeys = 100, numDups = 3, log, CompressionType.NONE, timestamp = T0, startValue = 0, step = 1)
+    writeKeyDups(numKeys = 100, numDups = 3, log, Compression.NONE, timestamp = T0, startValue = 0, step = 1)
 
     val startSizeBlock0 = log.size
 
@@ -159,7 +160,7 @@ class LogCleanerIntegrationTest extends AbstractLogCleanerIntegrationTest {
     val T1 = time.milliseconds
 
     // write the second block of data: all zero keys
-    val appends1 = writeKeyDups(numKeys = 100, numDups = 1, log, CompressionType.NONE, timestamp = T1, startValue = 0, step = 0)
+    val appends1 = writeKeyDups(numKeys = 100, numDups = 1, log, Compression.NONE, timestamp = T1, startValue = 0, step = 0)
 
     // roll the active segment
     log.roll()
@@ -199,7 +200,7 @@ class LogCleanerIntegrationTest extends AbstractLogCleanerIntegrationTest {
     }
   }
 
-  private def writeKeyDups(numKeys: Int, numDups: Int, log: UnifiedLog, codec: CompressionType, timestamp: Long,
+  private def writeKeyDups(numKeys: Int, numDups: Int, log: UnifiedLog, codec: Compression, timestamp: Long,
                            startValue: Int, step: Int): Seq[(Int, Int)] = {
     var valCounter = startValue
     for (_ <- 0 until numDups; key <- 0 until numKeys) yield {
diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerLagIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerLagIntegrationTest.scala
index 4ccbb3880a4..1cb35e10bb5 100644
--- a/core/src/test/scala/unit/kafka/log/LogCleanerLagIntegrationTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogCleanerLagIntegrationTest.scala
@@ -19,6 +19,7 @@ package kafka.log
 
 import kafka.utils._
 import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.compress.Compression
 import org.apache.kafka.common.record.CompressionType
 import org.apache.kafka.server.util.MockTime
 import org.junit.jupiter.api.Assertions._
@@ -45,7 +46,8 @@ class LogCleanerLagIntegrationTest extends AbstractLogCleanerIntegrationTest wit
 
   @ParameterizedTest
   @MethodSource(Array("parameters"))
-  def cleanerTest(codec: CompressionType): Unit = {
+  def cleanerTest(compressionType: CompressionType): Unit = {
+    val codec: Compression = Compression.of(compressionType).build()
     cleaner = makeCleaner(partitions = topicPartitions,
       backoffMs = cleanerBackOffMs,
       minCompactionLagMs = minCompactionLag,
@@ -102,7 +104,7 @@ class LogCleanerLagIntegrationTest extends AbstractLogCleanerIntegrationTest wit
     }
   }
 
-  private def writeDups(numKeys: Int, numDups: Int, log: UnifiedLog, codec: CompressionType, timestamp: Long): Seq[(Int, Int)] = {
+  private def writeDups(numKeys: Int, numDups: Int, log: UnifiedLog, codec: Compression, timestamp: Long): Seq[(Int, Int)] = {
     for (_ <- 0 until numDups; key <- 0 until numKeys) yield {
       val count = counter
       log.appendAsLeader(TestUtils.singletonRecords(value = counter.toString.getBytes, codec = codec,
diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala
index 0c01419f211..f17c724066f 100644
--- a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala
@@ -23,6 +23,7 @@ import java.util.Properties
 import kafka.server.BrokerTopicStats
 import kafka.utils._
 import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.compress.Compression
 import org.apache.kafka.common.config.TopicConfig
 import org.apache.kafka.common.record._
 import org.apache.kafka.common.utils.Utils
@@ -667,10 +668,10 @@ class LogCleanerManagerTest extends Logging {
     val producerId = 15L
     val producerEpoch = 0.toShort
     val sequence = 0
-    log.appendAsLeader(MemoryRecords.withTransactionalRecords(CompressionType.NONE, producerId, producerEpoch, sequence,
+    log.appendAsLeader(MemoryRecords.withTransactionalRecords(Compression.NONE, producerId, producerEpoch, sequence,
       new SimpleRecord(time.milliseconds(), "1".getBytes, "a".getBytes),
       new SimpleRecord(time.milliseconds(), "2".getBytes, "b".getBytes)), leaderEpoch = 0)
-    log.appendAsLeader(MemoryRecords.withTransactionalRecords(CompressionType.NONE, producerId, producerEpoch, sequence + 2,
+    log.appendAsLeader(MemoryRecords.withTransactionalRecords(Compression.NONE, producerId, producerEpoch, sequence + 2,
       new SimpleRecord(time.milliseconds(), "3".getBytes, "c".getBytes)), leaderEpoch = 0)
     log.roll()
     log.updateHighWatermark(3L)
@@ -853,7 +854,7 @@ class LogCleanerManagerTest extends Logging {
       new SimpleRecord(currentTimestamp, s"key-$offset".getBytes, s"value-$offset".getBytes)
     }
 
-    log.appendAsLeader(MemoryRecords.withRecords(CompressionType.NONE, records:_*), leaderEpoch = 1)
+    log.appendAsLeader(MemoryRecords.withRecords(Compression.NONE, records:_*), leaderEpoch = 1)
     log.maybeIncrementHighWatermark(log.logEndOffsetMetadata)
   }
 
@@ -876,6 +877,6 @@ class LogCleanerManagerTest extends Logging {
   }
 
   private def records(key: Int, value: Int, timestamp: Long) =
-    MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord(timestamp, key.toString.getBytes, value.toString.getBytes))
+    MemoryRecords.withRecords(Compression.NONE, new SimpleRecord(timestamp, key.toString.getBytes, value.toString.getBytes))
 
 }
diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala
index 226744f3431..876082dd074 100755
--- a/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogCleanerParameterizedIntegrationTest.scala
@@ -23,6 +23,7 @@ import kafka.server.KafkaConfig
 import kafka.server.checkpoints.OffsetCheckpointFile
 import kafka.utils._
 import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.compress.Compression
 import org.apache.kafka.common.config.TopicConfig
 import org.apache.kafka.common.record._
 import org.apache.kafka.server.common.MetadataVersion.{IBP_0_10_0_IV1, IBP_0_11_0_IV0, IBP_0_9_0}
@@ -48,7 +49,8 @@ class LogCleanerParameterizedIntegrationTest extends AbstractLogCleanerIntegrati
 
   @ParameterizedTest
   @ArgumentsSource(classOf[LogCleanerParameterizedIntegrationTest.AllCompressions])
-  def cleanerTest(codec: CompressionType): Unit = {
+  def cleanerTest(compressionType: CompressionType): Unit = {
+    val codec: Compression = Compression.of(compressionType).build()
     val largeMessageKey = 20
     val (largeMessageValue, largeMessageSet) = createLargeSingleMessageSet(largeMessageKey, RecordBatch.CURRENT_MAGIC_VALUE, codec)
     val maxMessageSize = largeMessageSet.sizeInBytes
@@ -91,7 +93,7 @@ class LogCleanerParameterizedIntegrationTest extends AbstractLogCleanerIntegrati
 
   @ParameterizedTest
   @ArgumentsSource(classOf[LogCleanerParameterizedIntegrationTest.AllCompressions])
-  def testCleansCombinedCompactAndDeleteTopic(codec: CompressionType): Unit = {
+  def testCleansCombinedCompactAndDeleteTopic(compressionType: CompressionType): Unit = {
     val logProps  = new Properties()
     val retentionMs: Integer = 100000
     logProps.put(TopicConfig.RETENTION_MS_CONFIG, retentionMs: Integer)
@@ -101,7 +103,7 @@ class LogCleanerParameterizedIntegrationTest extends AbstractLogCleanerIntegrati
       cleaner = makeCleaner(partitions = topicPartitions.take(1), propertyOverrides = logProps, backoffMs = 100L)
       val log = cleaner.logs.get(topicPartitions(0))
 
-      val messages = writeDups(numKeys = numKeys, numDups = 3, log = log, codec = codec)
+      val messages = writeDups(numKeys = numKeys, numDups = 3, log = log, codec = Compression.of(compressionType).build())
       val startSize = log.size
 
       log.updateHighWatermark(log.logEndOffset)
@@ -136,11 +138,12 @@ class LogCleanerParameterizedIntegrationTest extends AbstractLogCleanerIntegrati
   @nowarn("cat=deprecation")
   @ParameterizedTest
   @ArgumentsSource(classOf[LogCleanerParameterizedIntegrationTest.ExcludeZstd])
-  def testCleanerWithMessageFormatV0(codec: CompressionType): Unit = {
+  def testCleanerWithMessageFormatV0(compressionType: CompressionType): Unit = {
+    val codec: Compression = Compression.of(compressionType).build()
     val largeMessageKey = 20
     val (largeMessageValue, largeMessageSet) = createLargeSingleMessageSet(largeMessageKey, RecordBatch.MAGIC_VALUE_V0, codec)
     val maxMessageSize = codec match {
-      case CompressionType.NONE => largeMessageSet.sizeInBytes
+      case Compression.NONE => largeMessageSet.sizeInBytes
       case _ =>
         // the broker assigns absolute offsets for message format 0 which potentially causes the compressed size to
         // increase because the broker offsets are larger than the ones assigned by the client
@@ -191,6 +194,7 @@ class LogCleanerParameterizedIntegrationTest extends AbstractLogCleanerIntegrati
   @ParameterizedTest
   @ArgumentsSource(classOf[LogCleanerParameterizedIntegrationTest.ExcludeZstd])
   def testCleaningNestedMessagesWithV0AndV1(codec: CompressionType): Unit = {
+    val compression = Compression.of(codec).build()
     val maxMessageSize = 192
     cleaner = makeCleaner(partitions = topicPartitions, maxMessageSize = maxMessageSize, segmentSize = 256)
 
@@ -201,15 +205,15 @@ class LogCleanerParameterizedIntegrationTest extends AbstractLogCleanerIntegrati
 
     // with compression enabled, these messages will be written as a single message containing
     // all of the individual messages
-    var appendsV0 = writeDupsSingleMessageSet(numKeys = 2, numDups = 3, log = log, codec = codec, magicValue = RecordBatch.MAGIC_VALUE_V0)
-    appendsV0 ++= writeDupsSingleMessageSet(numKeys = 2, startKey = 3, numDups = 2, log = log, codec = codec, magicValue = RecordBatch.MAGIC_VALUE_V0)
+    var appendsV0 = writeDupsSingleMessageSet(numKeys = 2, numDups = 3, log = log, codec = compression, magicValue = RecordBatch.MAGIC_VALUE_V0)
+    appendsV0 ++= writeDupsSingleMessageSet(numKeys = 2, startKey = 3, numDups = 2, log = log, codec = compression, magicValue = RecordBatch.MAGIC_VALUE_V0)
 
     props.put(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG, IBP_0_10_0_IV1.version)
     log.updateConfig(new LogConfig(props))
 
-    var appendsV1 = writeDupsSingleMessageSet(startKey = 4, numKeys = 2, numDups = 2, log = log, codec = codec, magicValue = RecordBatch.MAGIC_VALUE_V1)
-    appendsV1 ++= writeDupsSingleMessageSet(startKey = 4, numKeys = 2, numDups = 2, log = log, codec = codec, magicValue = RecordBatch.MAGIC_VALUE_V1)
-    appendsV1 ++= writeDupsSingleMessageSet(startKey = 6, numKeys = 2, numDups = 2, log = log, codec = codec, magicValue = RecordBatch.MAGIC_VALUE_V1)
+    var appendsV1 = writeDupsSingleMessageSet(startKey = 4, numKeys = 2, numDups = 2, log = log, codec = compression, magicValue = RecordBatch.MAGIC_VALUE_V1)
+    appendsV1 ++= writeDupsSingleMessageSet(startKey = 4, numKeys = 2, numDups = 2, log = log, codec = compression, magicValue = RecordBatch.MAGIC_VALUE_V1)
+    appendsV1 ++= writeDupsSingleMessageSet(startKey = 6, numKeys = 2, numDups = 2, log = log, codec = compression, magicValue = RecordBatch.MAGIC_VALUE_V1)
 
     val appends = appendsV0 ++ appendsV1
 
@@ -228,7 +232,8 @@ class LogCleanerParameterizedIntegrationTest extends AbstractLogCleanerIntegrati
 
   @ParameterizedTest
   @ArgumentsSource(classOf[LogCleanerParameterizedIntegrationTest.AllCompressions])
-  def cleanerConfigUpdateTest(codec: CompressionType): Unit = {
+  def cleanerConfigUpdateTest(compressionType: CompressionType): Unit = {
+    val codec: Compression = Compression.of(compressionType).build()
     val largeMessageKey = 20
     val (_, largeMessageSet) = createLargeSingleMessageSet(largeMessageKey, RecordBatch.CURRENT_MAGIC_VALUE, codec)
     val maxMessageSize = largeMessageSet.sizeInBytes
@@ -305,7 +310,7 @@ class LogCleanerParameterizedIntegrationTest extends AbstractLogCleanerIntegrati
     }
   }
 
-  private def writeDupsSingleMessageSet(numKeys: Int, numDups: Int, log: UnifiedLog, codec: CompressionType,
+  private def writeDupsSingleMessageSet(numKeys: Int, numDups: Int, log: UnifiedLog, codec: Compression,
                                         startKey: Int = 0, magicValue: Byte): Seq[(Int, String, Long)] = {
     val kvs = for (_ <- 0 until numDups; key <- startKey until (startKey + numKeys)) yield {
       val payload = counter.toString
diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala
index 8afed829bd3..7e72a4852ca 100644
--- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala
@@ -21,6 +21,7 @@ import kafka.common._
 import kafka.server.{BrokerTopicStats, KafkaConfig}
 import kafka.utils._
 import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.compress.Compression
 import org.apache.kafka.common.config.TopicConfig
 import org.apache.kafka.common.errors.CorruptRecordException
 import org.apache.kafka.common.record._
@@ -1116,7 +1117,7 @@ class LogCleanerTest extends Logging {
 
     val producerId2 = 2L
     val records = MemoryRecords.withTransactionalRecords(
-      CompressionType.NONE,
+      Compression.NONE,
       producerId2,
       producerEpoch,
       0,
@@ -1993,7 +1994,7 @@ class LogCleanerTest extends Logging {
 
   private def invalidCleanedMessage(initialOffset: Long,
                                     keysAndValues: Iterable[(Int, Int)],
-                                    codec: CompressionType = CompressionType.GZIP): MemoryRecords = {
+                                    compressionType: CompressionType = CompressionType.GZIP): MemoryRecords = {
     // this function replicates the old versions of the cleaner which under some circumstances
     // would write invalid compressed message sets with the outer magic set to 1 and the inner
     // magic set to 0
@@ -2004,6 +2005,7 @@ class LogCleanerTest extends Logging {
         kv._2.toString.getBytes))
 
     val buffer = ByteBuffer.allocate(math.min(math.max(records.map(_.sizeInBytes()).sum / 2, 1024), 1 << 16))
+    val codec: Compression = Compression.of(compressionType).build()
     val builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V1, codec, TimestampType.CREATE_TIME, initialOffset)
 
     var offset = initialOffset
@@ -2016,7 +2018,7 @@ class LogCleanerTest extends Logging {
   }
 
   private def messageWithOffset(key: Array[Byte], value: Array[Byte], offset: Long): MemoryRecords =
-    MemoryRecords.withRecords(offset, CompressionType.NONE, 0, new SimpleRecord(key, value))
+    MemoryRecords.withRecords(offset, Compression.NONE, 0, new SimpleRecord(key, value))
 
   private def messageWithOffset(key: Int, value: Int, offset: Long): MemoryRecords =
     messageWithOffset(key.toString.getBytes, value.toString.getBytes, offset)
@@ -2061,7 +2063,7 @@ class LogCleanerTest extends Logging {
              producerEpoch: Short = RecordBatch.NO_PRODUCER_EPOCH,
              sequence: Int = RecordBatch.NO_SEQUENCE,
              partitionLeaderEpoch: Int = RecordBatch.NO_PARTITION_LEADER_EPOCH): MemoryRecords = {
-    MemoryRecords.withIdempotentRecords(RecordBatch.CURRENT_MAGIC_VALUE, 0L, CompressionType.NONE, producerId, producerEpoch, sequence,
+    MemoryRecords.withIdempotentRecords(RecordBatch.CURRENT_MAGIC_VALUE, 0L, Compression.NONE, producerId, producerEpoch, sequence,
       partitionLeaderEpoch, new SimpleRecord(key.toString.getBytes, value.toString.getBytes))
   }
 
@@ -2097,9 +2099,9 @@ class LogCleanerTest extends Logging {
         new SimpleRecord(time.milliseconds(), keyBytes, keyBytes) // the value doesn't matter since we validate offsets
       }
       val records = if (isTransactional)
-        MemoryRecords.withTransactionalRecords(CompressionType.NONE, producerId, producerEpoch, sequence, simpleRecords.toArray: _*)
+        MemoryRecords.withTransactionalRecords(Compression.NONE, producerId, producerEpoch, sequence, simpleRecords.toArray: _*)
       else
-        MemoryRecords.withIdempotentRecords(CompressionType.NONE, producerId, producerEpoch, sequence, simpleRecords.toArray: _*)
+        MemoryRecords.withIdempotentRecords(Compression.NONE, producerId, producerEpoch, sequence, simpleRecords.toArray: _*)
       sequence += simpleRecords.size
       log.appendAsLeader(records, leaderEpoch, origin)
     }
diff --git a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala
index d6982986e23..1b13225cc14 100644
--- a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala
@@ -95,6 +95,9 @@ class LogConfigTest {
       case TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG => assertPropertyInvalid(name, "not_a_boolean")
       case TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG => assertPropertyInvalid(name, "not_a_number", "-3")
       case TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG => assertPropertyInvalid(name, "not_a_number", "-3")
+      case TopicConfig.COMPRESSION_GZIP_LEVEL_CONFIG => assertPropertyInvalid(name, "not_a_number", "-2")
+      case TopicConfig.COMPRESSION_LZ4_LEVEL_CONFIG => assertPropertyInvalid(name, "not_a_number", "-1")
+      case TopicConfig.COMPRESSION_ZSTD_LEVEL_CONFIG => assertPropertyInvalid(name, "not_a_number", "-0.1")
 
       case _ => assertPropertyInvalid(name, "not_a_number", "-1")
     })
@@ -215,7 +218,7 @@ class LogConfigTest {
     values.foreach(value => {
       val props = new Properties
       props.setProperty(name, value.toString)
-      assertThrows(classOf[Exception], () => new LogConfig(props))
+      assertThrows(classOf[Exception], () => new LogConfig(props), () => s"Property $name should not allow $value")
     })
   }
 
diff --git a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala
index 02dbf35e440..e1c204bf871 100644
--- a/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogLoaderTest.scala
@@ -25,9 +25,10 @@ import kafka.server.{BrokerTopicStats, KafkaConfig}
 import kafka.server.metadata.MockConfigRepository
 import kafka.utils.TestUtils
 import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.compress.Compression
 import org.apache.kafka.common.config.TopicConfig
 import org.apache.kafka.common.errors.KafkaStorageException
-import org.apache.kafka.common.record.{CompressionType, ControlRecordType, DefaultRecordBatch, MemoryRecords, RecordBatch, RecordVersion, SimpleRecord, TimestampType}
+import org.apache.kafka.common.record.{ControlRecordType, DefaultRecordBatch, MemoryRecords, RecordBatch, RecordVersion, SimpleRecord, TimestampType}
 import org.apache.kafka.common.utils.{Time, Utils}
 import org.apache.kafka.coordinator.transaction.TransactionLogConfigs
 import org.apache.kafka.server.common.MetadataVersion
@@ -294,7 +295,7 @@ class LogLoaderTest {
                                               key: Array[Byte] = null,
                                               leaderEpoch: Int,
                                               offset: Long,
-                                              codec: CompressionType = CompressionType.NONE,
+                                              codec: Compression = Compression.NONE,
                                               timestamp: Long = RecordBatch.NO_TIMESTAMP,
                                               magicValue: Byte = RecordBatch.CURRENT_MAGIC_VALUE): MemoryRecords = {
     val records = Seq(new SimpleRecord(timestamp, key, value))
@@ -1063,10 +1064,10 @@ class LogLoaderTest {
     // append some messages to create some segments
     val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024)
     val log = createLog(logDir, logConfig)
-    val set1 = MemoryRecords.withRecords(0, CompressionType.NONE, 0, new SimpleRecord("v1".getBytes(), "k1".getBytes()))
-    val set2 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 2, CompressionType.NONE, 0, new SimpleRecord("v3".getBytes(), "k3".getBytes()))
-    val set3 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 3, CompressionType.NONE, 0, new SimpleRecord("v4".getBytes(), "k4".getBytes()))
-    val set4 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 4, CompressionType.NONE, 0, new SimpleRecord("v5".getBytes(), "k5".getBytes()))
+    val set1 = MemoryRecords.withRecords(0, Compression.NONE, 0, new SimpleRecord("v1".getBytes(), "k1".getBytes()))
+    val set2 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 2, Compression.NONE, 0, new SimpleRecord("v3".getBytes(), "k3".getBytes()))
+    val set3 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 3, Compression.NONE, 0, new SimpleRecord("v4".getBytes(), "k4".getBytes()))
+    val set4 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 4, Compression.NONE, 0, new SimpleRecord("v5".getBytes(), "k5".getBytes()))
     //Writes into an empty log with baseOffset 0
     log.appendAsFollower(set1)
     assertEquals(0L, log.activeSegment.baseOffset)
@@ -1120,14 +1121,14 @@ class LogLoaderTest {
     // append some messages to create some segments
     val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024)
     val log = createLog(logDir, logConfig)
-    val set1 = MemoryRecords.withRecords(0, CompressionType.NONE, 0, new SimpleRecord("v1".getBytes(), "k1".getBytes()))
-    val set2 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 2, CompressionType.GZIP, 0,
+    val set1 = MemoryRecords.withRecords(0, Compression.NONE, 0, new SimpleRecord("v1".getBytes(), "k1".getBytes()))
+    val set2 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 2, Compression.gzip().build(), 0,
       new SimpleRecord("v3".getBytes(), "k3".getBytes()),
       new SimpleRecord("v4".getBytes(), "k4".getBytes()))
-    val set3 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 4, CompressionType.GZIP, 0,
+    val set3 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 4, Compression.gzip().build(), 0,
       new SimpleRecord("v5".getBytes(), "k5".getBytes()),
       new SimpleRecord("v6".getBytes(), "k6".getBytes()))
-    val set4 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 6, CompressionType.GZIP, 0,
+    val set4 = MemoryRecords.withRecords(Integer.MAX_VALUE.toLong + 6, Compression.gzip().build(), 0,
       new SimpleRecord("v7".getBytes(), "k7".getBytes()),
       new SimpleRecord("v8".getBytes(), "k8".getBytes()))
     //Writes into an empty log with baseOffset 0
@@ -1159,15 +1160,15 @@ class LogLoaderTest {
     // append some messages to create some segments
     val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1000, indexIntervalBytes = 1, maxMessageBytes = 64 * 1024)
     val log = createLog(logDir, logConfig)
-    val set1 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, 0, CompressionType.NONE,
+    val set1 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, 0, Compression.NONE,
       new SimpleRecord("v1".getBytes(), "k1".getBytes()))
-    val set2 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, Integer.MAX_VALUE.toLong + 2, CompressionType.GZIP,
+    val set2 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, Integer.MAX_VALUE.toLong + 2, Compression.gzip().build(),
       new SimpleRecord("v3".getBytes(), "k3".getBytes()),
       new SimpleRecord("v4".getBytes(), "k4".getBytes()))
-    val set3 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, Integer.MAX_VALUE.toLong + 4, CompressionType.GZIP,
+    val set3 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, Integer.MAX_VALUE.toLong + 4, Compression.gzip().build(),
       new SimpleRecord("v5".getBytes(), "k5".getBytes()),
       new SimpleRecord("v6".getBytes(), "k6".getBytes()))
-    val set4 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, Integer.MAX_VALUE.toLong + 6, CompressionType.GZIP,
+    val set4 = MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, Integer.MAX_VALUE.toLong + 6, Compression.gzip().build(),
       new SimpleRecord("v7".getBytes(), "k7".getBytes()),
       new SimpleRecord("v8".getBytes(), "k8".getBytes()))
     //Writes into an empty log with baseOffset 0
diff --git a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala
index b545157d3c4..abfa4810160 100644
--- a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala
@@ -19,6 +19,7 @@ package kafka.log
 import kafka.utils.TestUtils
 import kafka.utils.TestUtils.random
 import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.compress.Compression
 import org.apache.kafka.common.config.TopicConfig
 import org.apache.kafka.common.record._
 import org.apache.kafka.common.utils.{MockTime, Time, Utils}
@@ -53,7 +54,7 @@ class LogSegmentTest {
 
   /* create a ByteBufferMessageSet for the given messages starting from the given offset */
   def records(offset: Long, records: String*): MemoryRecords = {
-    MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, offset, CompressionType.NONE, TimestampType.CREATE_TIME,
+    MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V1, offset, Compression.NONE, TimestampType.CREATE_TIME,
       records.map { s => new SimpleRecord(offset * 10, s.getBytes) }: _*)
   }
 
@@ -347,15 +348,15 @@ class LogSegmentTest {
 
     // append transactional records from pid1
     segment.append(101L, RecordBatch.NO_TIMESTAMP,
-      100L, MemoryRecords.withTransactionalRecords(100L, CompressionType.NONE,
+      100L, MemoryRecords.withTransactionalRecords(100L, Compression.NONE,
         pid1, producerEpoch, sequence, partitionLeaderEpoch, new SimpleRecord("a".getBytes), new SimpleRecord("b".getBytes)))
 
     // append transactional records from pid2
-    segment.append(103L, RecordBatch.NO_TIMESTAMP, 102L, MemoryRecords.withTransactionalRecords(102L, CompressionType.NONE,
+    segment.append(103L, RecordBatch.NO_TIMESTAMP, 102L, MemoryRecords.withTransactionalRecords(102L, Compression.NONE,
         pid2, producerEpoch, sequence, partitionLeaderEpoch, new SimpleRecord("a".getBytes), new SimpleRecord("b".getBytes)))
 
     // append non-transactional records
-    segment.append(105L, RecordBatch.NO_TIMESTAMP, 104L, MemoryRecords.withRecords(104L, CompressionType.NONE,
+    segment.append(105L, RecordBatch.NO_TIMESTAMP, 104L, MemoryRecords.withRecords(104L, Compression.NONE,
         partitionLeaderEpoch, new SimpleRecord("a".getBytes), new SimpleRecord("b".getBytes)))
 
     // abort the transaction from pid2 (note LSO should be 100L since the txn from pid1 has not completed)
@@ -413,16 +414,16 @@ class LogSegmentTest {
     }
 
     val cache = new LeaderEpochFileCache(topicPartition, checkpoint)
-    seg.append(105L, RecordBatch.NO_TIMESTAMP, 104L, MemoryRecords.withRecords(104L, CompressionType.NONE, 0,
+    seg.append(105L, RecordBatch.NO_TIMESTAMP, 104L, MemoryRecords.withRecords(104L, Compression.NONE, 0,
         new SimpleRecord("a".getBytes), new SimpleRecord("b".getBytes)))
 
-    seg.append(107L, RecordBatch.NO_TIMESTAMP, 106L, MemoryRecords.withRecords(106L, CompressionType.NONE, 1,
+    seg.append(107L, RecordBatch.NO_TIMESTAMP, 106L, MemoryRecords.withRecords(106L, Compression.NONE, 1,
         new SimpleRecord("a".getBytes), new SimpleRecord("b".getBytes)))
 
-    seg.append(109L, RecordBatch.NO_TIMESTAMP, 108L, MemoryRecords.withRecords(108L, CompressionType.NONE, 1,
+    seg.append(109L, RecordBatch.NO_TIMESTAMP, 108L, MemoryRecords.withRecords(108L, Compression.NONE, 1,
         new SimpleRecord("a".getBytes), new SimpleRecord("b".getBytes)))
 
-    seg.append(111L, RecordBatch.NO_TIMESTAMP, 110, MemoryRecords.withRecords(110L, CompressionType.NONE, 2,
+    seg.append(111L, RecordBatch.NO_TIMESTAMP, 110, MemoryRecords.withRecords(110L, Compression.NONE, 2,
         new SimpleRecord("a".getBytes), new SimpleRecord("b".getBytes)))
 
     seg.recover(newProducerStateManager(), Optional.of(cache))
@@ -555,7 +556,7 @@ class LogSegmentTest {
     val offset = 40
 
     def records(offset: Long, record: String): MemoryRecords =
-      MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, offset, CompressionType.NONE, TimestampType.CREATE_TIME,
+      MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, offset, Compression.NONE, TimestampType.CREATE_TIME,
         new SimpleRecord(offset * 1000, record.getBytes))
 
     //Given two messages with a gap between them (e.g. mid offset compacted away)
@@ -576,7 +577,7 @@ class LogSegmentTest {
   @Test
   def testAppendFromFile(): Unit = {
     def records(offset: Long, size: Int): MemoryRecords =
-      MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, offset, CompressionType.NONE, TimestampType.CREATE_TIME,
+      MemoryRecords.withRecords(RecordBatch.MAGIC_VALUE_V2, offset, Compression.NONE, TimestampType.CREATE_TIME,
         new SimpleRecord(new Array[Byte](size)))
 
     // create a log file in a separate directory to avoid conflicting with created segments
diff --git a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala
index e583f8cf6cb..706a5ec30f2 100644
--- a/core/src/test/scala/unit/kafka/log/LogTestUtils.scala
+++ b/core/src/test/scala/unit/kafka/log/LogTestUtils.scala
@@ -24,7 +24,8 @@ import java.util.Properties
 import kafka.server.BrokerTopicStats
 import kafka.utils.TestUtils
 import org.apache.kafka.common.Uuid
-import org.apache.kafka.common.record.{CompressionType, ControlRecordType, EndTransactionMarker, FileRecords, MemoryRecords, RecordBatch, SimpleRecord}
+import org.apache.kafka.common.compress.Compression
+import org.apache.kafka.common.record.{ControlRecordType, EndTransactionMarker, FileRecords, MemoryRecords, RecordBatch, SimpleRecord}
 import org.apache.kafka.common.utils.{Time, Utils}
 import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse}
 
@@ -156,12 +157,12 @@ object LogTestUtils {
         new SimpleRecord(data, data)
       }
 
-      segment.append(MemoryRecords.withRecords(baseOffset, CompressionType.NONE, 0,
+      segment.append(MemoryRecords.withRecords(baseOffset, Compression.NONE, 0,
         record(baseOffset)))
-      segment.append(MemoryRecords.withRecords(baseOffset + 1, CompressionType.NONE, 0,
+      segment.append(MemoryRecords.withRecords(baseOffset + 1, Compression.NONE, 0,
         record(baseOffset + 1),
         record(baseOffset + 2)))
-      segment.append(MemoryRecords.withRecords(baseOffset + Int.MaxValue - 1, CompressionType.NONE, 0,
+      segment.append(MemoryRecords.withRecords(baseOffset + Int.MaxValue - 1, Compression.NONE, 0,
         record(baseOffset + Int.MaxValue - 1)))
       // Need to create the offset files explicitly to avoid triggering segment recovery to truncate segment.
       Files.createFile(LogFileUtils.offsetIndexFile(logDir, baseOffset).toPath)
@@ -265,7 +266,7 @@ object LogTestUtils {
     val simpleRecords = (0 until numRecords).map { seq =>
       new SimpleRecord(s"$seq".getBytes)
     }
-    val records = MemoryRecords.withRecords(CompressionType.NONE, simpleRecords: _*)
+    val records = MemoryRecords.withRecords(Compression.NONE, simpleRecords: _*)
     log.appendAsLeader(records, leaderEpoch = 0)
   }
 
@@ -288,10 +289,10 @@ object LogTestUtils {
       }
 
       val records = if (isTransactional) {
-        MemoryRecords.withTransactionalRecords(CompressionType.NONE, producerId,
+        MemoryRecords.withTransactionalRecords(Compression.NONE, producerId,
           producerEpoch, sequence, simpleRecords: _*)
       } else {
-        MemoryRecords.withIdempotentRecords(CompressionType.NONE, producerId,
+        MemoryRecords.withIdempotentRecords(Compression.NONE, producerId,
           producerEpoch, sequence, simpleRecords: _*)
       }
 
diff --git a/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala b/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala
index 0462fb86753..c2b29bcc6d6 100644
--- a/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala
@@ -20,6 +20,7 @@ import java.nio.ByteBuffer
 import java.util.concurrent.TimeUnit
 import kafka.server.{BrokerTopicStats, RequestLocal}
 import kafka.utils.TestUtils.meterCount
+import org.apache.kafka.common.compress.{Compression, GzipCompression, Lz4Compression}
 import org.apache.kafka.common.errors.{InvalidTimestampException, UnsupportedCompressionTypeException, UnsupportedForMessageFormatException}
 import org.apache.kafka.common.record._
 import org.apache.kafka.common.utils.{PrimitiveRef, Time}
@@ -44,38 +45,39 @@ class LogValidatorTest {
 
   @Test
   def testOnlyOneBatch(): Unit = {
-    checkOnlyOneBatch(RecordBatch.MAGIC_VALUE_V0, CompressionType.GZIP, CompressionType.GZIP)
-    checkOnlyOneBatch(RecordBatch.MAGIC_VALUE_V1, CompressionType.GZIP, CompressionType.GZIP)
-    checkOnlyOneBatch(RecordBatch.MAGIC_VALUE_V2, CompressionType.GZIP, CompressionType.GZIP)
-    checkOnlyOneBatch(RecordBatch.MAGIC_VALUE_V0, CompressionType.GZIP, CompressionType.NONE)
-    checkOnlyOneBatch(RecordBatch.MAGIC_VALUE_V1, CompressionType.GZIP, CompressionType.NONE)
-    checkOnlyOneBatch(RecordBatch.MAGIC_VALUE_V2, CompressionType.GZIP, CompressionType.NONE)
-    checkOnlyOneBatch(RecordBatch.MAGIC_VALUE_V2, CompressionType.NONE, CompressionType.NONE)
-    checkOnlyOneBatch(RecordBatch.MAGIC_VALUE_V2, CompressionType.NONE, CompressionType.GZIP)
+    checkOnlyOneBatch(RecordBatch.MAGIC_VALUE_V0, Compression.gzip().build(), Compression.gzip().build())
+    checkOnlyOneBatch(RecordBatch.MAGIC_VALUE_V1, Compression.gzip().build(), Compression.gzip().build())
+    checkOnlyOneBatch(RecordBatch.MAGIC_VALUE_V2, Compression.gzip().build(), Compression.gzip().build())
+    checkOnlyOneBatch(RecordBatch.MAGIC_VALUE_V0, Compression.gzip().build(), Compression.NONE)
+    checkOnlyOneBatch(RecordBatch.MAGIC_VALUE_V1, Compression.gzip().build(), Compression.NONE)
+    checkOnlyOneBatch(RecordBatch.MAGIC_VALUE_V2, Compression.gzip().build(), Compression.NONE)
+    checkOnlyOneBatch(RecordBatch.MAGIC_VALUE_V2, Compression.NONE, Compression.NONE)
+    checkOnlyOneBatch(RecordBatch.MAGIC_VALUE_V2, Compression.NONE, Compression.gzip().build())
   }
 
   @Test
   def testAllowMultiBatch(): Unit = {
-    checkAllowMultiBatch(RecordBatch.MAGIC_VALUE_V0, CompressionType.NONE, CompressionType.NONE)
-    checkAllowMultiBatch(RecordBatch.MAGIC_VALUE_V1, CompressionType.NONE, CompressionType.NONE)
-    checkAllowMultiBatch(RecordBatch.MAGIC_VALUE_V0, CompressionType.NONE, CompressionType.GZIP)
-    checkAllowMultiBatch(RecordBatch.MAGIC_VALUE_V1, CompressionType.NONE, CompressionType.GZIP)
+    checkAllowMultiBatch(RecordBatch.MAGIC_VALUE_V0, Compression.NONE, Compression.NONE)
+    checkAllowMultiBatch(RecordBatch.MAGIC_VALUE_V1, Compression.NONE, Compression.NONE)
+    checkAllowMultiBatch(RecordBatch.MAGIC_VALUE_V0, Compression.NONE, Compression.gzip().build())
+    checkAllowMultiBatch(RecordBatch.MAGIC_VALUE_V1, Compression.NONE, Compression.gzip().build())
   }
 
   @Test
   def testValidationOfBatchesWithNonSequentialInnerOffsets(): Unit = {
     def testMessageValidation(magicValue: Byte): Unit = {
       val numRecords = 20
-      val invalidRecords = recordsWithNonSequentialInnerOffsets(magicValue, CompressionType.GZIP, numRecords)
+      val compression: Compression = Compression.gzip().build()
+      val invalidRecords = recordsWithNonSequentialInnerOffsets(magicValue, compression, numRecords)
 
       // Validation for v2 and above is strict for this case. For older formats, we fix invalid
       // internal offsets by rewriting the batch.
       if (magicValue >= RecordBatch.MAGIC_VALUE_V2) {
         assertThrows(classOf[InvalidRecordException],
-          () => validateMessages(invalidRecords, magicValue, CompressionType.GZIP, CompressionType.GZIP)
+          () => validateMessages(invalidRecords, magicValue, CompressionType.GZIP, compression)
         )
       } else {
-        val result = validateMessages(invalidRecords, magicValue, CompressionType.GZIP, CompressionType.GZIP)
+        val result = validateMessages(invalidRecords, magicValue, CompressionType.GZIP, compression)
         assertEquals(0 until numRecords, result.validatedRecords.records.asScala.map(_.offset))
       }
     }
@@ -87,23 +89,24 @@ class LogValidatorTest {
 
   @Test
   def testMisMatchMagic(): Unit = {
-    checkMismatchMagic(RecordBatch.MAGIC_VALUE_V0, RecordBatch.MAGIC_VALUE_V1, CompressionType.GZIP)
-    checkMismatchMagic(RecordBatch.MAGIC_VALUE_V1, RecordBatch.MAGIC_VALUE_V0, CompressionType.GZIP)
+    val compression: Compression = Compression.gzip().build()
+    checkMismatchMagic(RecordBatch.MAGIC_VALUE_V0, RecordBatch.MAGIC_VALUE_V1, compression)
+    checkMismatchMagic(RecordBatch.MAGIC_VALUE_V1, RecordBatch.MAGIC_VALUE_V0, compression)
   }
 
-  private def checkOnlyOneBatch(magic: Byte, sourceCompressionType: CompressionType, targetCompressionType: CompressionType): Unit = {
+  private def checkOnlyOneBatch(magic: Byte, sourceCompression: Compression, targetCompression: Compression): Unit = {
     assertThrows(classOf[InvalidRecordException],
-      () => validateMessages(createTwoBatchedRecords(magic, 0L, sourceCompressionType), magic, sourceCompressionType, targetCompressionType)
+      () => validateMessages(createTwoBatchedRecords(magic, sourceCompression), magic, sourceCompression.`type`(), targetCompression)
     )
   }
 
-  private def checkAllowMultiBatch(magic: Byte, sourceCompressionType: CompressionType, targetCompressionType: CompressionType): Unit = {
-    validateMessages(createTwoBatchedRecords(magic, 0L, sourceCompressionType), magic, sourceCompressionType, targetCompressionType)
+  private def checkAllowMultiBatch(magic: Byte, sourceCompression: Compression, targetCompression: Compression): Unit = {
+    validateMessages(createTwoBatchedRecords(magic, sourceCompression), magic, sourceCompression.`type`(), targetCompression)
   }
 
-  private def checkMismatchMagic(batchMagic: Byte, recordMagic: Byte, compressionType: CompressionType): Unit = {
+  private def checkMismatchMagic(batchMagic: Byte, recordMagic: Byte, compression: Compression): Unit = {
     assertThrows(classOf[RecordValidationException],
-      () => validateMessages(recordsWithInvalidInnerMagic(batchMagic, recordMagic, compressionType), batchMagic, compressionType, compressionType)
+      () => validateMessages(recordsWithInvalidInnerMagic(batchMagic, recordMagic, compression), batchMagic, compression.`type`(), compression)
     )
     assertEquals(metricsKeySet.count(_.getMBeanName.endsWith(s"${BrokerTopicStats.InvalidMagicNumberRecordsPerSec}")), 1)
     assertTrue(meterCount(s"${BrokerTopicStats.InvalidMagicNumberRecordsPerSec}") > 0)
@@ -112,13 +115,13 @@ class LogValidatorTest {
   private def validateMessages(records: MemoryRecords,
                                magic: Byte,
                                sourceCompressionType: CompressionType,
-                               targetCompressionType: CompressionType): ValidationResult = {
+                               targetCompression: Compression): ValidationResult = {
     val mockTime = new MockTime(0L, 0L)
     new LogValidator(records,
       topicPartition,
       mockTime,
       sourceCompressionType,
-      targetCompressionType,
+      targetCompression,
       false,
       magic,
       TimestampType.CREATE_TIME,
@@ -150,13 +153,13 @@ class LogValidatorTest {
   private def checkLogAppendTimeNonCompressed(magic: Byte): Unit = {
     val mockTime = new MockTime
     // The timestamps should be overwritten
-    val records = createRecords(magicValue = magic, timestamp = 1234L, codec = CompressionType.NONE)
+    val records = createRecords(magicValue = magic, timestamp = 1234L, codec = Compression.NONE)
     val offsetCounter = PrimitiveRef.ofLong(0)
     val validatedResults = new LogValidator(records,
       topicPartition,
       mockTime,
       CompressionType.NONE,
-      CompressionType.NONE,
+      Compression.NONE,
       false,
       magic,
       TimestampType.LOG_APPEND_TIME,
@@ -197,15 +200,16 @@ class LogValidatorTest {
   }
 
   private def checkLogAppendTimeWithRecompression(targetMagic: Byte): Unit = {
+    val compression: Compression = Compression.gzip().build()
     val mockTime = new MockTime
     // The timestamps should be overwritten
-    val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V0, codec = CompressionType.GZIP)
+    val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V0, codec = compression)
     val validatedResults = new LogValidator(
       records,
       topicPartition,
       mockTime,
       CompressionType.GZIP,
-      CompressionType.GZIP,
+      compression,
       false,
       targetMagic,
       TimestampType.LOG_APPEND_TIME,
@@ -249,15 +253,16 @@ class LogValidatorTest {
   }
 
   private def checkLogAppendTimeWithoutRecompression(magic: Byte): Unit = {
+    val compression: Compression = Compression.gzip().build()
     val mockTime = new MockTime
     // The timestamps should be overwritten
-    val records = createRecords(magicValue = magic, timestamp = 1234L, codec = CompressionType.GZIP)
+    val records = createRecords(magicValue = magic, timestamp = 1234L, codec = compression)
     val validatedResults = new LogValidator(
       records,
       topicPartition,
       mockTime,
       CompressionType.GZIP,
-      CompressionType.GZIP,
+      compression,
       false,
       magic,
       TimestampType.LOG_APPEND_TIME,
@@ -315,7 +320,7 @@ class LogValidatorTest {
   }
 
   private def validateRecordBatchWithCountOverrides(lastOffsetDelta: Int, count: Int): Unit = {
-    val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = 1234L, codec = CompressionType.NONE)
+    val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = 1234L, codec = Compression.NONE)
     records.buffer.putInt(DefaultRecordBatch.RECORDS_COUNT_OFFSET, count)
     records.buffer.putInt(DefaultRecordBatch.LAST_OFFSET_DELTA_OFFSET, lastOffsetDelta)
     new LogValidator(
@@ -323,7 +328,7 @@ class LogValidatorTest {
       topicPartition,
       time,
       CompressionType.GZIP,
-      CompressionType.GZIP,
+      Compression.gzip().build(),
       false,
       RecordBatch.MAGIC_VALUE_V2,
       TimestampType.LOG_APPEND_TIME,
@@ -367,7 +372,7 @@ class LogValidatorTest {
       new SimpleRecord(timestampSeq(2), "beautiful".getBytes)
     )
 
-    val records = MemoryRecords.withRecords(magic, 0L, CompressionType.NONE, TimestampType.CREATE_TIME, producerId,
+    val records = MemoryRecords.withRecords(magic, 0L, Compression.NONE, TimestampType.CREATE_TIME, producerId,
       producerEpoch, baseSequence, partitionLeaderEpoch, isTransactional, recordList: _*)
 
     val offsetCounter = PrimitiveRef.ofLong(0)
@@ -375,7 +380,7 @@ class LogValidatorTest {
       topicPartition,
       time,
       CompressionType.NONE,
-      CompressionType.NONE,
+      Compression.NONE,
       false,
       magic,
       TimestampType.CREATE_TIME,
@@ -452,7 +457,7 @@ class LogValidatorTest {
         (RecordBatch.NO_PRODUCER_ID, RecordBatch.NO_PRODUCER_EPOCH, RecordBatch.NO_SEQUENCE, false,
           RecordBatch.NO_PARTITION_LEADER_EPOCH)
 
-    val records = MemoryRecords.withRecords(magic, 0L, CompressionType.NONE, TimestampType.CREATE_TIME, producerId,
+    val records = MemoryRecords.withRecords(magic, 0L, Compression.NONE, TimestampType.CREATE_TIME, producerId,
       producerEpoch, baseSequence, partitionLeaderEpoch, isTransactional,
       new SimpleRecord(timestampSeq(0), "hello".getBytes),
       new SimpleRecord(timestampSeq(1), "there".getBytes),
@@ -465,7 +470,7 @@ class LogValidatorTest {
       topicPartition,
       time,
       CompressionType.NONE,
-      CompressionType.GZIP,
+      Compression.gzip().build(),
       false,
       magic,
       TimestampType.CREATE_TIME,
@@ -522,12 +527,13 @@ class LogValidatorTest {
   }
 
   private def checkCreateTimeUpConversionFromV0(toMagic: Byte): Unit = {
-    val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V0, codec = CompressionType.GZIP)
+    val compression: Compression = Compression.gzip().build()
+    val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V0, codec = compression)
     val validatedResults = new LogValidator(records,
       topicPartition,
       time,
       CompressionType.GZIP,
-      CompressionType.GZIP,
+      compression,
       false,
       toMagic,
       TimestampType.CREATE_TIME,
@@ -569,12 +575,13 @@ class LogValidatorTest {
   @Test
   def testCreateTimeUpConversionV1ToV2(): Unit = {
     val timestamp = System.currentTimeMillis()
-    val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V1, codec = CompressionType.GZIP, timestamp = timestamp)
+    val compression: Compression = Compression.gzip().build()
+    val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V1, codec = compression, timestamp = timestamp)
     val validatedResults = new LogValidator(records,
       topicPartition,
       time,
       CompressionType.GZIP,
-      CompressionType.GZIP,
+      compression,
       false,
       RecordBatch.MAGIC_VALUE_V2,
       TimestampType.CREATE_TIME,
@@ -631,14 +638,14 @@ class LogValidatorTest {
       new SimpleRecord(timestampSeq(2), "beautiful".getBytes)
     )
 
-    val records = MemoryRecords.withRecords(magic, 0L, CompressionType.GZIP, TimestampType.CREATE_TIME, producerId,
+    val records = MemoryRecords.withRecords(magic, 0L, Compression.gzip().build(), TimestampType.CREATE_TIME, producerId,
       producerEpoch, baseSequence, partitionLeaderEpoch, isTransactional, recordList: _*)
 
     val validatedResults = new LogValidator(records,
       topicPartition,
       time,
       CompressionType.GZIP,
-      CompressionType.GZIP,
+      Compression.gzip().build(),
       false,
       magic,
       TimestampType.CREATE_TIME,
@@ -690,13 +697,13 @@ class LogValidatorTest {
   def testInvalidCreateTimeNonCompressedV1(): Unit = {
     val now = System.currentTimeMillis()
     val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V1, timestamp = now - 1001L,
-      codec = CompressionType.NONE)
+      codec = Compression.NONE)
     assertThrows(classOf[RecordValidationException], () => new LogValidator(
       records,
       topicPartition,
       time,
       CompressionType.NONE,
-      CompressionType.NONE,
+      Compression.NONE,
       false,
       RecordBatch.MAGIC_VALUE_V1,
       TimestampType.CREATE_TIME,
@@ -716,13 +723,13 @@ class LogValidatorTest {
   def testInvalidCreateTimeNonCompressedV2(): Unit = {
     val now = System.currentTimeMillis()
     val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = now - 1001L,
-      codec = CompressionType.NONE)
+      codec = Compression.NONE)
     assertThrows(classOf[RecordValidationException], () => new LogValidator(
       records,
       topicPartition,
       time,
       CompressionType.NONE,
-      CompressionType.NONE,
+      Compression.NONE,
       false,
       RecordBatch.MAGIC_VALUE_V2,
       TimestampType.CREATE_TIME,
@@ -741,14 +748,15 @@ class LogValidatorTest {
   @Test
   def testInvalidCreateTimeCompressedV1(): Unit = {
     val now = System.currentTimeMillis()
+    val compression: Compression = Compression.gzip().build()
     val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V1, timestamp = now - 1001L,
-      codec = CompressionType.GZIP)
+      codec = compression)
     assertThrows(classOf[RecordValidationException], () => new LogValidator(
       records,
       topicPartition,
       time,
       CompressionType.GZIP,
-      CompressionType.GZIP,
+      compression,
       false,
       RecordBatch.MAGIC_VALUE_V1,
       TimestampType.CREATE_TIME,
@@ -767,14 +775,15 @@ class LogValidatorTest {
   @Test
   def testInvalidCreateTimeCompressedV2(): Unit = {
     val now = System.currentTimeMillis()
+    val compression: Compression = Compression.gzip().build()
     val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = now - 1001L,
-      codec = CompressionType.GZIP)
+      codec = compression)
     assertThrows(classOf[RecordValidationException], () => new LogValidator(
       records,
       topicPartition,
       time,
       CompressionType.GZIP,
-      CompressionType.GZIP,
+      compression,
       false,
       RecordBatch.MAGIC_VALUE_V1,
       TimestampType.CREATE_TIME,
@@ -792,14 +801,14 @@ class LogValidatorTest {
 
   @Test
   def testAbsoluteOffsetAssignmentNonCompressed(): Unit = {
-    val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V0, codec = CompressionType.NONE)
+    val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V0, codec = Compression.NONE)
     val offset = 1234567
     checkOffsets(records, 0)
     checkOffsets(new LogValidator(records,
       topicPartition,
       time,
       CompressionType.NONE,
-      CompressionType.NONE,
+      Compression.NONE,
       false,
       RecordBatch.MAGIC_VALUE_V0,
       TimestampType.CREATE_TIME,
@@ -815,14 +824,15 @@ class LogValidatorTest {
 
   @Test
   def testAbsoluteOffsetAssignmentCompressed(): Unit = {
-    val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V0, codec = CompressionType.GZIP)
+    val compression: Compression = Compression.gzip().build()
+    val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V0, codec = compression)
     val offset = 1234567
     checkOffsets(records, 0)
     checkOffsets(new LogValidator(records,
       topicPartition,
       time,
       CompressionType.GZIP,
-      CompressionType.GZIP,
+      compression,
       false,
       RecordBatch.MAGIC_VALUE_V0,
       TimestampType.CREATE_TIME,
@@ -839,14 +849,14 @@ class LogValidatorTest {
   @Test
   def testRelativeOffsetAssignmentNonCompressedV1(): Unit = {
     val now = System.currentTimeMillis()
-    val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V1, timestamp = now, codec = CompressionType.NONE)
+    val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V1, timestamp = now, codec = Compression.NONE)
     val offset = 1234567
     checkOffsets(records, 0)
     val messageWithOffset = new LogValidator(records,
       topicPartition,
       time,
       CompressionType.NONE,
-      CompressionType.NONE,
+      Compression.NONE,
       false,
       RecordBatch.MAGIC_VALUE_V1,
       TimestampType.CREATE_TIME,
@@ -864,14 +874,14 @@ class LogValidatorTest {
   @Test
   def testRelativeOffsetAssignmentNonCompressedV2(): Unit = {
     val now = System.currentTimeMillis()
-    val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = now, codec = CompressionType.NONE)
+    val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = now, codec = Compression.NONE)
     val offset = 1234567
     checkOffsets(records, 0)
     val messageWithOffset = new LogValidator(records,
       topicPartition,
       time,
       CompressionType.NONE,
-      CompressionType.NONE,
+      Compression.NONE,
       false,
       RecordBatch.MAGIC_VALUE_V2,
       TimestampType.CREATE_TIME,
@@ -889,7 +899,8 @@ class LogValidatorTest {
   @Test
   def testRelativeOffsetAssignmentCompressedV1(): Unit = {
     val now = System.currentTimeMillis()
-    val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V1, timestamp = now, codec = CompressionType.GZIP)
+    val compression: Compression = Compression.gzip().build()
+    val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V1, timestamp = now, codec = compression)
     val offset = 1234567
     checkOffsets(records, 0)
     val compressedMessagesWithOffset = new LogValidator(
@@ -897,7 +908,7 @@ class LogValidatorTest {
       topicPartition,
       time,
       CompressionType.GZIP,
-      CompressionType.GZIP,
+      compression,
       false,
       RecordBatch.MAGIC_VALUE_V1,
       TimestampType.CREATE_TIME,
@@ -915,7 +926,8 @@ class LogValidatorTest {
   @Test
   def testRelativeOffsetAssignmentCompressedV2(): Unit = {
     val now = System.currentTimeMillis()
-    val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = now, codec = CompressionType.GZIP)
+    val compression: Compression = Compression.gzip().build()
+    val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = now, codec = compression)
     val offset = 1234567
     checkOffsets(records, 0)
     val compressedMessagesWithOffset = new LogValidator(
@@ -923,7 +935,7 @@ class LogValidatorTest {
       topicPartition,
       time,
       CompressionType.GZIP,
-      CompressionType.GZIP,
+      compression,
       false,
       RecordBatch.MAGIC_VALUE_V2,
       TimestampType.CREATE_TIME,
@@ -940,14 +952,14 @@ class LogValidatorTest {
 
   @Test
   def testOffsetAssignmentAfterUpConversionV0ToV1NonCompressed(): Unit = {
-    val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V0, codec = CompressionType.NONE)
+    val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V0, codec = Compression.NONE)
     checkOffsets(records, 0)
     val offset = 1234567
     val validatedResults = new LogValidator(records,
       topicPartition,
       time,
       CompressionType.NONE,
-      CompressionType.NONE,
+      Compression.NONE,
       false,
       RecordBatch.MAGIC_VALUE_V1,
       TimestampType.LOG_APPEND_TIME,
@@ -966,14 +978,14 @@ class LogValidatorTest {
 
   @Test
   def testOffsetAssignmentAfterUpConversionV0ToV2NonCompressed(): Unit = {
-    val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V0, codec = CompressionType.NONE)
+    val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V0, codec = Compression.NONE)
     checkOffsets(records, 0)
     val offset = 1234567
     val validatedResults = new LogValidator(records,
       topicPartition,
       time,
       CompressionType.NONE,
-      CompressionType.NONE,
+      Compression.NONE,
       false,
       RecordBatch.MAGIC_VALUE_V2,
       TimestampType.LOG_APPEND_TIME,
@@ -992,14 +1004,15 @@ class LogValidatorTest {
 
   @Test
   def testOffsetAssignmentAfterUpConversionV0ToV1Compressed(): Unit = {
-    val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V0, codec = CompressionType.GZIP)
+    val compression: Compression = Compression.gzip().build()
+    val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V0, codec = compression)
     val offset = 1234567
     checkOffsets(records, 0)
     val validatedResults = new LogValidator(records,
       topicPartition,
       time,
       CompressionType.GZIP,
-      CompressionType.GZIP,
+      compression,
       false,
       RecordBatch.MAGIC_VALUE_V1,
       TimestampType.LOG_APPEND_TIME,
@@ -1018,14 +1031,15 @@ class LogValidatorTest {
 
   @Test
   def testOffsetAssignmentAfterUpConversionV0ToV2Compressed(): Unit = {
-    val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V0, codec = CompressionType.GZIP)
+    val compression: Compression = Compression.gzip().build()
+    val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V0, codec = compression)
     val offset = 1234567
     checkOffsets(records, 0)
     val validatedResults = new LogValidator(records,
       topicPartition,
       time,
       CompressionType.GZIP,
-      CompressionType.GZIP,
+      compression,
       false,
       RecordBatch.MAGIC_VALUE_V2,
       TimestampType.LOG_APPEND_TIME,
@@ -1051,7 +1065,7 @@ class LogValidatorTest {
       topicPartition,
       time,
       CompressionType.NONE,
-      CompressionType.NONE,
+      Compression.NONE,
       false,
       RecordBatch.CURRENT_MAGIC_VALUE,
       TimestampType.CREATE_TIME,
@@ -1074,7 +1088,7 @@ class LogValidatorTest {
       topicPartition,
       time,
       CompressionType.NONE,
-      CompressionType.SNAPPY,
+      Compression.snappy().build(),
       false,
       RecordBatch.CURRENT_MAGIC_VALUE,
       TimestampType.CREATE_TIME,
@@ -1096,13 +1110,13 @@ class LogValidatorTest {
   def testOffsetAssignmentAfterDownConversionV1ToV0NonCompressed(): Unit = {
     val offset = 1234567
     val now = System.currentTimeMillis()
-    val records = createRecords(RecordBatch.MAGIC_VALUE_V1, now, codec = CompressionType.NONE)
+    val records = createRecords(RecordBatch.MAGIC_VALUE_V1, now, codec = Compression.NONE)
     checkOffsets(records, 0)
     checkOffsets(new LogValidator(records,
       topicPartition,
       time,
       CompressionType.NONE,
-      CompressionType.NONE,
+      Compression.NONE,
       false,
       RecordBatch.MAGIC_VALUE_V0,
       TimestampType.CREATE_TIME,
@@ -1120,13 +1134,14 @@ class LogValidatorTest {
   def testOffsetAssignmentAfterDownConversionV1ToV0Compressed(): Unit = {
     val offset = 1234567
     val now = System.currentTimeMillis()
-    val records = createRecords(RecordBatch.MAGIC_VALUE_V1, now, CompressionType.GZIP)
+    val compression: Compression = Compression.gzip().build()
+    val records = createRecords(RecordBatch.MAGIC_VALUE_V1, now, compression)
     checkOffsets(records, 0)
     checkOffsets(new LogValidator(records,
       topicPartition,
       time,
       CompressionType.GZIP,
-      CompressionType.GZIP,
+      compression,
       false,
       RecordBatch.MAGIC_VALUE_V0,
       TimestampType.CREATE_TIME,
@@ -1142,14 +1157,14 @@ class LogValidatorTest {
 
   @Test
   def testOffsetAssignmentAfterUpConversionV1ToV2NonCompressed(): Unit = {
-    val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V1, codec = CompressionType.NONE)
+    val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V1, codec = Compression.NONE)
     checkOffsets(records, 0)
     val offset = 1234567
     checkOffsets(new LogValidator(records,
       topicPartition,
       time,
       CompressionType.NONE,
-      CompressionType.NONE,
+      Compression.NONE,
       false,
       RecordBatch.MAGIC_VALUE_V2,
       TimestampType.LOG_APPEND_TIME,
@@ -1165,14 +1180,15 @@ class LogValidatorTest {
 
   @Test
   def testOffsetAssignmentAfterUpConversionV1ToV2Compressed(): Unit = {
-    val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V1, codec = CompressionType.GZIP)
+    val compression: Compression = Compression.gzip().build()
+    val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V1, codec = compression)
     val offset = 1234567
     checkOffsets(records, 0)
     checkOffsets(new LogValidator(records,
       topicPartition,
       time,
       CompressionType.GZIP,
-      CompressionType.GZIP,
+      compression,
       false,
       RecordBatch.MAGIC_VALUE_V2,
       TimestampType.LOG_APPEND_TIME,
@@ -1190,13 +1206,13 @@ class LogValidatorTest {
   def testOffsetAssignmentAfterDownConversionV2ToV1NonCompressed(): Unit = {
     val offset = 1234567
     val now = System.currentTimeMillis()
-    val records = createRecords(RecordBatch.MAGIC_VALUE_V2, now, codec = CompressionType.NONE)
+    val records = createRecords(RecordBatch.MAGIC_VALUE_V2, now, codec = Compression.NONE)
     checkOffsets(records, 0)
     checkOffsets(new LogValidator(records,
       topicPartition,
       time,
       CompressionType.NONE,
-      CompressionType.NONE,
+      Compression.NONE,
       false,
       RecordBatch.MAGIC_VALUE_V1,
       TimestampType.CREATE_TIME,
@@ -1214,13 +1230,14 @@ class LogValidatorTest {
   def testOffsetAssignmentAfterDownConversionV2ToV1Compressed(): Unit = {
     val offset = 1234567
     val now = System.currentTimeMillis()
-    val records = createRecords(RecordBatch.MAGIC_VALUE_V2, now, CompressionType.GZIP)
+    val compression: Compression = Compression.gzip().build()
+    val records = createRecords(RecordBatch.MAGIC_VALUE_V2, now, compression)
     checkOffsets(records, 0)
     checkOffsets(new LogValidator(records,
       topicPartition,
       time,
       CompressionType.GZIP,
-      CompressionType.GZIP,
+      compression,
       false,
       RecordBatch.MAGIC_VALUE_V1,
       TimestampType.CREATE_TIME,
@@ -1240,13 +1257,13 @@ class LogValidatorTest {
     val producerId = 1344L
     val producerEpoch = 16.toShort
     val sequence = 0
-    val records = MemoryRecords.withTransactionalRecords(CompressionType.NONE, producerId, producerEpoch, sequence,
+    val records = MemoryRecords.withTransactionalRecords(Compression.NONE, producerId, producerEpoch, sequence,
       new SimpleRecord("hello".getBytes), new SimpleRecord("there".getBytes), new SimpleRecord("beautiful".getBytes))
     assertThrows(classOf[UnsupportedForMessageFormatException], () => new LogValidator(records,
       topicPartition,
       time,
       CompressionType.GZIP,
-      CompressionType.GZIP,
+      Compression.gzip().build(),
       false,
       RecordBatch.MAGIC_VALUE_V1,
       TimestampType.CREATE_TIME,
@@ -1266,13 +1283,13 @@ class LogValidatorTest {
     val producerId = 1344L
     val producerEpoch = 16.toShort
     val sequence = 0
-    val records = MemoryRecords.withIdempotentRecords(CompressionType.NONE, producerId, producerEpoch, sequence,
+    val records = MemoryRecords.withIdempotentRecords(Compression.NONE, producerId, producerEpoch, sequence,
       new SimpleRecord("hello".getBytes), new SimpleRecord("there".getBytes), new SimpleRecord("beautiful".getBytes))
     assertThrows(classOf[UnsupportedForMessageFormatException], () => new LogValidator(records,
       topicPartition,
       time,
       CompressionType.GZIP,
-      CompressionType.GZIP,
+      Compression.gzip().build(),
       false,
       RecordBatch.MAGIC_VALUE_V1,
       TimestampType.CREATE_TIME,
@@ -1290,13 +1307,13 @@ class LogValidatorTest {
   def testOffsetAssignmentAfterDownConversionV2ToV0NonCompressed(): Unit = {
     val offset = 1234567
     val now = System.currentTimeMillis()
-    val records = createRecords(RecordBatch.MAGIC_VALUE_V2, now, codec = CompressionType.NONE)
+    val records = createRecords(RecordBatch.MAGIC_VALUE_V2, now, codec = Compression.NONE)
     checkOffsets(records, 0)
     checkOffsets(new LogValidator(records,
       topicPartition,
       time,
       CompressionType.NONE,
-      CompressionType.NONE,
+      Compression.NONE,
       false,
       RecordBatch.MAGIC_VALUE_V0,
       TimestampType.CREATE_TIME,
@@ -1314,13 +1331,14 @@ class LogValidatorTest {
   def testOffsetAssignmentAfterDownConversionV2ToV0Compressed(): Unit = {
     val offset = 1234567
     val now = System.currentTimeMillis()
-    val records = createRecords(RecordBatch.MAGIC_VALUE_V2, now, CompressionType.GZIP)
+    val compression: Compression = Compression.gzip().build()
+    val records = createRecords(RecordBatch.MAGIC_VALUE_V2, now, compression)
     checkOffsets(records, 0)
     checkOffsets(new LogValidator(records,
       topicPartition,
       time,
       CompressionType.GZIP,
-      CompressionType.GZIP,
+      compression,
       false,
       RecordBatch.MAGIC_VALUE_V0,
       TimestampType.CREATE_TIME,
@@ -1342,12 +1360,12 @@ class LogValidatorTest {
         topicPartition,
         time,
         CompressionType.GZIP,
-        CompressionType.GZIP,
+        Compression.gzip().build(),
         false,
         RecordBatch.MAGIC_VALUE_V0,
         TimestampType.CREATE_TIME,
         5000L,
-      5000L,
+        5000L,
         RecordBatch.NO_PARTITION_LEADER_EPOCH,
         AppendOrigin.CLIENT,
         MetadataVersion.latestTesting
@@ -1360,18 +1378,18 @@ class LogValidatorTest {
 
   @Test
   def testCompressedBatchWithoutRecordsNotAllowed(): Unit = {
-    testBatchWithoutRecordsNotAllowed(CompressionType.GZIP, CompressionType.GZIP)
+    testBatchWithoutRecordsNotAllowed(CompressionType.GZIP, Compression.gzip().build())
   }
 
   @Test
   def testZStdCompressedWithUnavailableIBPVersion(): Unit = {
     // The timestamps should be overwritten
-    val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = 1234L, codec = CompressionType.NONE)
+    val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = 1234L, codec = Compression.NONE)
     assertThrows(classOf[UnsupportedCompressionTypeException], () => new LogValidator(records,
       topicPartition,
       time,
       CompressionType.NONE,
-      CompressionType.ZSTD,
+      Compression.zstd().build(),
       false,
       RecordBatch.MAGIC_VALUE_V2,
       TimestampType.LOG_APPEND_TIME,
@@ -1387,26 +1405,27 @@ class LogValidatorTest {
 
   @Test
   def testUncompressedBatchWithoutRecordsNotAllowed(): Unit = {
-    testBatchWithoutRecordsNotAllowed(CompressionType.NONE, CompressionType.NONE)
+    testBatchWithoutRecordsNotAllowed(CompressionType.NONE, Compression.NONE)
   }
 
   @Test
   def testRecompressedBatchWithoutRecordsNotAllowed(): Unit = {
-    testBatchWithoutRecordsNotAllowed(CompressionType.NONE, CompressionType.GZIP)
+    testBatchWithoutRecordsNotAllowed(CompressionType.NONE, Compression.gzip().build())
   }
 
   @Test
   def testInvalidTimestampExceptionHasBatchIndex(): Unit = {
     val now = System.currentTimeMillis()
+    val compression: Compression = Compression.gzip().build()
     val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = now - 1001L,
-      codec = CompressionType.GZIP)
+      codec = compression)
     val e = assertThrows(classOf[RecordValidationException],
       () => new LogValidator(
         records,
         topicPartition,
         time,
         CompressionType.GZIP,
-        CompressionType.GZIP,
+        compression,
         false,
         RecordBatch.MAGIC_VALUE_V1,
         TimestampType.CREATE_TIME,
@@ -1428,9 +1447,12 @@ class LogValidatorTest {
   @Test
   def testInvalidRecordExceptionHasBatchIndex(): Unit = {
     val e = assertThrows(classOf[RecordValidationException],
-      () => validateMessages(recordsWithInvalidInnerMagic(
-        RecordBatch.MAGIC_VALUE_V0, RecordBatch.MAGIC_VALUE_V1, CompressionType.GZIP),
-        RecordBatch.MAGIC_VALUE_V0, CompressionType.GZIP, CompressionType.GZIP)
+      () => {
+        val compression: Compression = Compression.gzip().build()
+        validateMessages(recordsWithInvalidInnerMagic(
+          RecordBatch.MAGIC_VALUE_V0, RecordBatch.MAGIC_VALUE_V1, compression),
+          RecordBatch.MAGIC_VALUE_V0, CompressionType.GZIP, compression)
+      }
     )
 
     assertTrue(e.invalidException.isInstanceOf[InvalidRecordException])
@@ -1442,12 +1464,13 @@ class LogValidatorTest {
 
   @Test
   def testBatchWithInvalidRecordsAndInvalidTimestamp(): Unit = {
+    val compression: Compression = Compression.gzip().build()
     val records = (0 until 5).map(id =>
       LegacyRecord.create(RecordBatch.MAGIC_VALUE_V0, 0L, null, id.toString.getBytes())
     )
 
     val buffer = ByteBuffer.allocate(1024)
-    val builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V1, CompressionType.GZIP,
+    val builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V1, compression,
       TimestampType.CREATE_TIME, 0L)
     var offset = 0
 
@@ -1462,7 +1485,7 @@ class LogValidatorTest {
 
     val e = assertThrows(classOf[RecordValidationException],
       () => validateMessages(invalidOffsetTimestampRecords,
-        RecordBatch.MAGIC_VALUE_V0, CompressionType.GZIP, CompressionType.GZIP)
+        RecordBatch.MAGIC_VALUE_V0, CompressionType.GZIP, compression)
     )
     // if there is a mix of both regular InvalidRecordException and InvalidTimestampException,
     // InvalidTimestampException takes precedence
@@ -1477,15 +1500,16 @@ class LogValidatorTest {
     val timestampAfterMaxConfig = 1 * 60 * 60 * 1000L //1 hr
     val now = System.currentTimeMillis()
     val fiveMinutesBeforeThreshold = now - timestampBeforeMaxConfig - (5 * 60 * 1000L)
+    val compression: Compression = Compression.gzip().build()
     val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = fiveMinutesBeforeThreshold,
-      codec = CompressionType.GZIP)
+      codec = compression)
     val e = assertThrows(classOf[RecordValidationException],
       () => new LogValidator(
         records,
         topicPartition,
         time,
         CompressionType.GZIP,
-        CompressionType.GZIP,
+        compression,
         false,
         RecordBatch.MAGIC_VALUE_V2,
         TimestampType.CREATE_TIME,
@@ -1511,15 +1535,16 @@ class LogValidatorTest {
     val timestampAfterMaxConfig = 1 * 60 * 60 * 1000L //1 hr
     val now = System.currentTimeMillis()
     val fiveMinutesAfterThreshold = now + timestampAfterMaxConfig + (5 * 60 * 1000L)
+    val compression: Compression = Compression.gzip().build()
     val records = createRecords(magicValue = RecordBatch.MAGIC_VALUE_V2, timestamp = fiveMinutesAfterThreshold,
-      codec = CompressionType.GZIP)
+      codec = compression)
     val e = assertThrows(classOf[RecordValidationException],
       () => new LogValidator(
         records,
         topicPartition,
         time,
         CompressionType.GZIP,
-        CompressionType.GZIP,
+        compression,
         false,
         RecordBatch.MAGIC_VALUE_V2,
         TimestampType.CREATE_TIME,
@@ -1538,7 +1563,80 @@ class LogValidatorTest {
     assertEquals(e.recordErrors.size, 3)
   }
 
-  private def testBatchWithoutRecordsNotAllowed(sourceCompression: CompressionType, targetCompression: CompressionType): Unit = {
+  @Test
+  def testDifferentLevelDoesNotCauseRecompression(): Unit = {
+    val records = List(
+      List.fill(256)("some").mkString("").getBytes,
+      List.fill(256)("data").mkString("").getBytes
+    )
+    // Records from the producer were created with gzip max level
+    val gzipMax: Compression = Compression.gzip().level(GzipCompression.MAX_LEVEL).build()
+    val recordsGzipMax = createRecords(records, RecordBatch.MAGIC_VALUE_V2, RecordBatch.NO_TIMESTAMP, gzipMax)
+
+    // The topic is configured with gzip min level
+    val gzipMin: Compression = Compression.gzip().level(GzipCompression.MIN_LEVEL).build()
+    val recordsGzipMin = createRecords(records, RecordBatch.MAGIC_VALUE_V2, RecordBatch.NO_TIMESTAMP, gzipMin)
+
+    // ensure data compressed with gzip max and min is different
+    assertNotEquals(recordsGzipMax, recordsGzipMin)
+    val validator = new LogValidator(recordsGzipMax,
+      topicPartition,
+      time,
+      gzipMax.`type`(),
+      gzipMin,
+      false,
+      RecordBatch.MAGIC_VALUE_V2,
+      TimestampType.CREATE_TIME,
+      5000L,
+      5000L,
+      RecordBatch.NO_PARTITION_LEADER_EPOCH,
+      AppendOrigin.CLIENT,
+      MetadataVersion.latestTesting
+    )
+    val result = validator.validateMessagesAndAssignOffsets(
+      PrimitiveRef.ofLong(0L), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier
+    )
+    // ensure validated records have not been changed so they are the same as the producer records
+    assertEquals(recordsGzipMax, result.validatedRecords)
+    assertNotEquals(recordsGzipMin, result.validatedRecords)
+  }
+
+  @Test
+  def testDifferentCodecCausesRecompression(): Unit = {
+    val records = List(
+      List.fill(256)("some").mkString("").getBytes,
+      List.fill(256)("data").mkString("").getBytes
+    )
+    // Records from the producer were created with gzip max level
+    val gzipMax: Compression = Compression.gzip().level(GzipCompression.MAX_LEVEL).build()
+    val recordsGzipMax = createRecords(records, RecordBatch.MAGIC_VALUE_V2, RecordBatch.NO_TIMESTAMP, gzipMax)
+
+    // The topic is configured with lz4 min level
+    val lz4Min: Compression = Compression.lz4().level(Lz4Compression.MIN_LEVEL).build()
+    val recordsLz4Min = createRecords(records, RecordBatch.MAGIC_VALUE_V2, RecordBatch.NO_TIMESTAMP, lz4Min)
+
+    val validator = new LogValidator(recordsGzipMax,
+      topicPartition,
+      time,
+      gzipMax.`type`(),
+      lz4Min,
+      false,
+      RecordBatch.MAGIC_VALUE_V2,
+      TimestampType.CREATE_TIME,
+      5000L,
+      5000L,
+      RecordBatch.NO_PARTITION_LEADER_EPOCH,
+      AppendOrigin.CLIENT,
+      MetadataVersion.latestTesting
+    )
+    val result = validator.validateMessagesAndAssignOffsets(
+      PrimitiveRef.ofLong(0L), metricsRecorder, RequestLocal.withThreadConfinedCaching.bufferSupplier
+    )
+    // ensure validated records have been recompressed and match lz4 min level
+    assertEquals(recordsLz4Min, result.validatedRecords)
+  }
+
+  private def testBatchWithoutRecordsNotAllowed(sourceCompression: CompressionType, targetCompression: Compression): Unit = {
     val offset = 1234567
     val (producerId, producerEpoch, baseSequence, isTransactional, partitionLeaderEpoch) =
       (1324L, 10.toShort, 984, true, 40)
@@ -1568,18 +1666,26 @@ class LogValidatorTest {
 
   private def createRecords(magicValue: Byte,
                             timestamp: Long = RecordBatch.NO_TIMESTAMP,
-                            codec: CompressionType): MemoryRecords = {
+                            codec: Compression): MemoryRecords = {
+    val records = List("hello".getBytes, "there".getBytes, "beautiful".getBytes)
+    createRecords(records = records, magicValue = magicValue, timestamp = timestamp, codec = codec)
+  }
+
+  private def createRecords(records: List[Array[Byte]],
+                            magicValue: Byte,
+                            timestamp: Long,
+                            codec: Compression): MemoryRecords = {
     val buf = ByteBuffer.allocate(512)
     val builder = MemoryRecords.builder(buf, magicValue, codec, TimestampType.CREATE_TIME, 0L)
-    builder.appendWithOffset(0, timestamp, null, "hello".getBytes)
-    builder.appendWithOffset(1, timestamp, null, "there".getBytes)
-    builder.appendWithOffset(2, timestamp, null, "beautiful".getBytes)
+    records.indices.foreach { offset =>
+      builder.appendWithOffset(offset, timestamp, null, records(offset))
+    }
     builder.build()
   }
 
   private def createNonIncreasingOffsetRecords(magicValue: Byte,
                                                timestamp: Long = RecordBatch.NO_TIMESTAMP,
-                                               codec: CompressionType = CompressionType.NONE): MemoryRecords = {
+                                               codec: Compression = Compression.NONE): MemoryRecords = {
     val buf = ByteBuffer.allocate(512)
     val builder = MemoryRecords.builder(buf, magicValue, codec, TimestampType.CREATE_TIME, 0L)
     builder.appendWithOffset(0, timestamp, null, "hello".getBytes)
@@ -1588,9 +1694,7 @@ class LogValidatorTest {
     builder.build()
   }
 
-  private def createTwoBatchedRecords(magicValue: Byte,
-                                      timestamp: Long,
-                                      codec: CompressionType): MemoryRecords = {
+  private def createTwoBatchedRecords(magicValue: Byte, codec: Compression): MemoryRecords = {
     val buf = ByteBuffer.allocate(2048)
     var builder = MemoryRecords.builder(buf, magicValue, codec, TimestampType.CREATE_TIME, 0L)
     builder.append(10L, "1".getBytes(), "a".getBytes())
@@ -1615,7 +1719,7 @@ class LogValidatorTest {
   }
 
   private def recordsWithNonSequentialInnerOffsets(magicValue: Byte,
-                                                   compression: CompressionType,
+                                                   compression: Compression,
                                                    numRecords: Int): MemoryRecords = {
     val records = (0 until numRecords).map { id =>
       new SimpleRecord(id.toString.getBytes)
@@ -1633,7 +1737,7 @@ class LogValidatorTest {
 
   private def recordsWithInvalidInnerMagic(batchMagicValue: Byte,
                                            recordMagicValue: Byte,
-                                           codec: CompressionType): MemoryRecords = {
+                                           codec: Compression): MemoryRecords = {
     val records = (0 until 20).map(id =>
       LegacyRecord.create(recordMagicValue,
         RecordBatch.NO_TIMESTAMP,
diff --git a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala
index c52e9a666b3..77b63bf89ac 100755
--- a/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala
+++ b/core/src/test/scala/unit/kafka/log/UnifiedLogTest.scala
@@ -21,6 +21,7 @@ import kafka.common.{OffsetsOutOfOrderException, UnexpectedAppendOffsetException
 import kafka.log.remote.RemoteLogManager
 import kafka.server.{BrokerTopicStats, KafkaConfig}
 import kafka.utils._
+import org.apache.kafka.common.compress.Compression
 import org.apache.kafka.common.config.TopicConfig
 import org.apache.kafka.common.{InvalidRecordException, TopicPartition, Uuid}
 import org.apache.kafka.common.errors._
@@ -200,7 +201,7 @@ class UnifiedLogTest {
     )), leaderEpoch = 0)
 
     log.appendAsLeader(MemoryRecords.withTransactionalRecords(
-      CompressionType.NONE,
+      Compression.NONE,
       producerId,
       producerEpoch,
       sequence,
@@ -243,7 +244,7 @@ class UnifiedLogTest {
     )), leaderEpoch = 0)
 
     log.appendAsLeader(MemoryRecords.withTransactionalRecords(
-      CompressionType.NONE,
+      Compression.NONE,
       producerId,
       producerEpoch,
       sequence,
@@ -567,7 +568,7 @@ class UnifiedLogTest {
 
     val numSegments = log.numberOfSegments
     mockTime.sleep(log.config.segmentMs + 1)
-    log.appendAsLeader(MemoryRecords.withRecords(CompressionType.NONE), leaderEpoch = 0)
+    log.appendAsLeader(MemoryRecords.withRecords(Compression.NONE), leaderEpoch = 0)
     assertEquals(numSegments, log.numberOfSegments, "Appending an empty message set should not roll log even if sufficient time has passed.")
   }
 
@@ -1205,7 +1206,7 @@ class UnifiedLogTest {
     val seq = 0
 
     // add some transactional records
-    val records = MemoryRecords.withTransactionalRecords(CompressionType.NONE, pid, epoch, seq,
+    val records = MemoryRecords.withTransactionalRecords(Compression.NONE, pid, epoch, seq,
       new SimpleRecord("foo".getBytes),
       new SimpleRecord("bar".getBytes),
       new SimpleRecord("baz".getBytes))
@@ -1328,22 +1329,22 @@ class UnifiedLogTest {
     val epoch: Short = 0
     val buffer = ByteBuffer.allocate(512)
 
-    var builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, CompressionType.NONE,
+    var builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, Compression.NONE,
       TimestampType.LOG_APPEND_TIME, 0L, mockTime.milliseconds(), 1L, epoch, 0, false, 0)
     builder.append(new SimpleRecord("key".getBytes, "value".getBytes))
     builder.close()
 
-    builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, CompressionType.NONE,
+    builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, Compression.NONE,
       TimestampType.LOG_APPEND_TIME, 1L, mockTime.milliseconds(), 2L, epoch, 0, false, 0)
     builder.append(new SimpleRecord("key".getBytes, "value".getBytes))
     builder.close()
 
-    builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, CompressionType.NONE,
+    builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, Compression.NONE,
       TimestampType.LOG_APPEND_TIME, 2L, mockTime.milliseconds(), 3L, epoch, 0, false, 0)
     builder.append(new SimpleRecord("key".getBytes, "value".getBytes))
     builder.close()
 
-    builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, CompressionType.NONE,
+    builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, Compression.NONE,
       TimestampType.LOG_APPEND_TIME, 3L, mockTime.milliseconds(), 4L, epoch, 0, false, 0)
     builder.append(new SimpleRecord("key".getBytes, "value".getBytes))
     builder.close()
@@ -1378,9 +1379,9 @@ class UnifiedLogTest {
     // this is a bit contrived. to trigger the duplicate case for a follower append, we have to append
     // a batch with matching sequence numbers, but valid increasing offsets
     assertEquals(0L, log.logEndOffset)
-    log.appendAsFollower(MemoryRecords.withIdempotentRecords(0L, CompressionType.NONE, pid, epoch, baseSequence,
+    log.appendAsFollower(MemoryRecords.withIdempotentRecords(0L, Compression.NONE, pid, epoch, baseSequence,
       partitionLeaderEpoch, new SimpleRecord("a".getBytes), new SimpleRecord("b".getBytes)))
-    log.appendAsFollower(MemoryRecords.withIdempotentRecords(2L, CompressionType.NONE, pid, epoch, baseSequence,
+    log.appendAsFollower(MemoryRecords.withIdempotentRecords(2L, Compression.NONE, pid, epoch, baseSequence,
       partitionLeaderEpoch, new SimpleRecord("a".getBytes), new SimpleRecord("b".getBytes)))
 
     // Ensure that even the duplicate sequences are accepted on the follower.
@@ -1399,31 +1400,31 @@ class UnifiedLogTest {
     val buffer = ByteBuffer.allocate(512)
 
     // pid1 seq = 0
-    var builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, CompressionType.NONE,
+    var builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, Compression.NONE,
       TimestampType.LOG_APPEND_TIME, 0L, mockTime.milliseconds(), pid1, epoch, 0)
     builder.append(new SimpleRecord("key".getBytes, "value".getBytes))
     builder.close()
 
     // pid2 seq = 0
-    builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, CompressionType.NONE,
+    builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, Compression.NONE,
       TimestampType.LOG_APPEND_TIME, 1L, mockTime.milliseconds(), pid2, epoch, 0)
     builder.append(new SimpleRecord("key".getBytes, "value".getBytes))
     builder.close()
 
     // pid1 seq = 1
-    builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, CompressionType.NONE,
+    builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, Compression.NONE,
       TimestampType.LOG_APPEND_TIME, 2L, mockTime.milliseconds(), pid1, epoch, 1)
     builder.append(new SimpleRecord("key".getBytes, "value".getBytes))
     builder.close()
 
     // pid2 seq = 1
-    builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, CompressionType.NONE,
+    builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, Compression.NONE,
       TimestampType.LOG_APPEND_TIME, 3L, mockTime.milliseconds(), pid2, epoch, 1)
     builder.append(new SimpleRecord("key".getBytes, "value".getBytes))
     builder.close()
 
     // // pid1 seq = 1 (duplicate)
-    builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, CompressionType.NONE,
+    builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, Compression.NONE,
       TimestampType.LOG_APPEND_TIME, 4L, mockTime.milliseconds(), pid1, epoch, 1)
     builder.append(new SimpleRecord("key".getBytes, "value".getBytes))
     builder.close()
@@ -1578,7 +1579,7 @@ class UnifiedLogTest {
 
     // now test the case that we give the offsets and use non-sequential offsets
     for (i <- records.indices)
-      log.appendAsFollower(MemoryRecords.withRecords(messageIds(i), CompressionType.NONE, 0, records(i)))
+      log.appendAsFollower(MemoryRecords.withRecords(messageIds(i), Compression.NONE, 0, records(i)))
     for (i <- 50 until messageIds.max) {
       val idx = messageIds.indexWhere(_ >= i)
       val read = LogTestUtils.readLog(log, i, 100).records.records.iterator.next()
@@ -1626,7 +1627,7 @@ class UnifiedLogTest {
 
     // now test the case that we give the offsets and use non-sequential offsets
     for (i <- records.indices)
-      log.appendAsFollower(MemoryRecords.withRecords(messageIds(i), CompressionType.NONE, 0, records(i)))
+      log.appendAsFollower(MemoryRecords.withRecords(messageIds(i), Compression.NONE, 0, records(i)))
 
     for (i <- 50 until messageIds.max) {
       val idx = messageIds.indexWhere(_ >= i)
@@ -1651,7 +1652,7 @@ class UnifiedLogTest {
 
     // now test the case that we give the offsets and use non-sequential offsets
     for (i <- records.indices)
-      log.appendAsFollower(MemoryRecords.withRecords(messageIds(i), CompressionType.NONE, 0, records(i)))
+      log.appendAsFollower(MemoryRecords.withRecords(messageIds(i), Compression.NONE, 0, records(i)))
 
     for (i <- 50 until messageIds.max) {
       assertEquals(MemoryRecords.EMPTY, LogTestUtils.readLog(log, i, maxLength = 0, minOneMessage = false).records)
@@ -1752,8 +1753,8 @@ class UnifiedLogTest {
     val log = createLog(logDir, logConfig)
 
     /* append 2 compressed message sets, each with two messages giving offsets 0, 1, 2, 3 */
-    log.appendAsLeader(MemoryRecords.withRecords(CompressionType.GZIP, new SimpleRecord("hello".getBytes), new SimpleRecord("there".getBytes)), leaderEpoch = 0)
-    log.appendAsLeader(MemoryRecords.withRecords(CompressionType.GZIP, new SimpleRecord("alpha".getBytes), new SimpleRecord("beta".getBytes)), leaderEpoch = 0)
+    log.appendAsLeader(MemoryRecords.withRecords(Compression.gzip().build(), new SimpleRecord("hello".getBytes), new SimpleRecord("there".getBytes)), leaderEpoch = 0)
+    log.appendAsLeader(MemoryRecords.withRecords(Compression.gzip().build(), new SimpleRecord("alpha".getBytes), new SimpleRecord("beta".getBytes)), leaderEpoch = 0)
 
     def read(offset: Int) = LogTestUtils.readLog(log, offset, 4096).records.records
 
@@ -1807,7 +1808,7 @@ class UnifiedLogTest {
    */
   @Test
   def testMessageSetSizeCheck(): Unit = {
-    val messageSet = MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("You".getBytes), new SimpleRecord("bethe".getBytes))
+    val messageSet = MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("You".getBytes), new SimpleRecord("bethe".getBytes))
     // append messages to log
     val configSegmentSize = messageSet.sizeInBytes - 1
     val logConfig = LogTestUtils.createLogConfig(segmentBytes = configSegmentSize)
@@ -1822,13 +1823,13 @@ class UnifiedLogTest {
     val anotherKeyedMessage = new SimpleRecord("another key".getBytes, "this message also has a key".getBytes)
     val unkeyedMessage = new SimpleRecord("this message does not have a key".getBytes)
 
-    val messageSetWithUnkeyedMessage = MemoryRecords.withRecords(CompressionType.NONE, unkeyedMessage, keyedMessage)
-    val messageSetWithOneUnkeyedMessage = MemoryRecords.withRecords(CompressionType.NONE, unkeyedMessage)
-    val messageSetWithCompressedKeyedMessage = MemoryRecords.withRecords(CompressionType.GZIP, keyedMessage)
-    val messageSetWithCompressedUnkeyedMessage = MemoryRecords.withRecords(CompressionType.GZIP, keyedMessage, unkeyedMessage)
+    val messageSetWithUnkeyedMessage = MemoryRecords.withRecords(Compression.NONE, unkeyedMessage, keyedMessage)
+    val messageSetWithOneUnkeyedMessage = MemoryRecords.withRecords(Compression.NONE, unkeyedMessage)
+    val messageSetWithCompressedKeyedMessage = MemoryRecords.withRecords(Compression.gzip().build(), keyedMessage)
+    val messageSetWithCompressedUnkeyedMessage = MemoryRecords.withRecords(Compression.gzip().build(), keyedMessage, unkeyedMessage)
 
-    val messageSetWithKeyedMessage = MemoryRecords.withRecords(CompressionType.NONE, keyedMessage)
-    val messageSetWithKeyedMessages = MemoryRecords.withRecords(CompressionType.NONE, keyedMessage, anotherKeyedMessage)
+    val messageSetWithKeyedMessage = MemoryRecords.withRecords(Compression.NONE, keyedMessage)
+    val messageSetWithKeyedMessages = MemoryRecords.withRecords(Compression.NONE, keyedMessage, anotherKeyedMessage)
 
     val logConfig = LogTestUtils.createLogConfig(cleanupPolicy = TopicConfig.CLEANUP_POLICY_COMPACT)
     val log = createLog(logDir, logConfig)
@@ -1872,8 +1873,8 @@ class UnifiedLogTest {
    */
   @Test
   def testMessageSizeCheck(): Unit = {
-    val first = MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("You".getBytes), new SimpleRecord("bethe".getBytes))
-    val second = MemoryRecords.withRecords(CompressionType.NONE,
+    val first = MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("You".getBytes), new SimpleRecord("bethe".getBytes))
+    val second = MemoryRecords.withRecords(Compression.NONE,
       new SimpleRecord("change (I need more bytes)... blah blah blah.".getBytes),
       new SimpleRecord("More padding boo hoo".getBytes))
 
@@ -1891,9 +1892,9 @@ class UnifiedLogTest {
 
   @Test
   def testMessageSizeCheckInAppendAsFollower(): Unit = {
-    val first = MemoryRecords.withRecords(0, CompressionType.NONE, 0,
+    val first = MemoryRecords.withRecords(0, Compression.NONE, 0,
       new SimpleRecord("You".getBytes), new SimpleRecord("bethe".getBytes))
-    val second = MemoryRecords.withRecords(5, CompressionType.NONE, 0,
+    val second = MemoryRecords.withRecords(5, Compression.NONE, 0,
       new SimpleRecord("change (I need more bytes)... blah blah blah.".getBytes),
       new SimpleRecord("More padding boo hoo".getBytes))
 
@@ -1908,7 +1909,7 @@ class UnifiedLogTest {
   def testLogFlushesPartitionMetadataOnAppend(): Unit = {
     val logConfig = LogTestUtils.createLogConfig()
     val log = createLog(logDir, logConfig)
-    val record = MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("simpleValue".getBytes))
+    val record = MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("simpleValue".getBytes))
 
     val topicId = Uuid.randomUuid()
     log.partitionMetadataFile.get.record(topicId)
@@ -2000,7 +2001,7 @@ class UnifiedLogTest {
     val log = createLog(logDir, logConfig)
 
     val messages = (0 until numMessages).map { i =>
-      MemoryRecords.withRecords(100 + i, CompressionType.NONE, 0, new SimpleRecord(mockTime.milliseconds + i, i.toString.getBytes()))
+      MemoryRecords.withRecords(100 + i, Compression.NONE, 0, new SimpleRecord(mockTime.milliseconds + i, i.toString.getBytes()))
     }
     messages.foreach(log.appendAsFollower)
     val timeIndexEntries = log.logSegments.asScala.foldLeft(0) { (entries, segment) => entries + segment.timeIndex.entries }
@@ -2367,7 +2368,7 @@ class UnifiedLogTest {
     val appendOffsets = Seq(0L, 1L, 3L, 2L, 4L)
     val buffer = ByteBuffer.allocate(512)
     for (offset <- appendOffsets) {
-      val builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, CompressionType.NONE,
+      val builder = MemoryRecords.builder(buffer, RecordBatch.MAGIC_VALUE_V2, Compression.NONE,
                                           TimestampType.LOG_APPEND_TIME, offset, mockTime.milliseconds(),
                                           1L, 0, 0, false, 0)
       builder.append(new SimpleRecord("key".getBytes, "value".getBytes))
@@ -2385,15 +2386,16 @@ class UnifiedLogTest {
   def testAppendBelowExpectedOffsetThrowsException(): Unit = {
     val log = createLog(logDir, new LogConfig(new Properties))
     val records = (0 until 2).map(id => new SimpleRecord(id.toString.getBytes)).toArray
-    records.foreach(record => log.appendAsLeader(MemoryRecords.withRecords(CompressionType.NONE, record), leaderEpoch = 0))
+    records.foreach(record => log.appendAsLeader(MemoryRecords.withRecords(Compression.NONE, record), leaderEpoch = 0))
 
     val magicVals = Seq(RecordBatch.MAGIC_VALUE_V0, RecordBatch.MAGIC_VALUE_V1, RecordBatch.MAGIC_VALUE_V2)
     val compressionTypes = Seq(CompressionType.NONE, CompressionType.LZ4)
-    for (magic <- magicVals; compression <- compressionTypes) {
+    for (magic <- magicVals; compressionType <- compressionTypes) {
+      val compression = Compression.of(compressionType).build()
       val invalidRecord = MemoryRecords.withRecords(magic, compression, new SimpleRecord(1.toString.getBytes))
       assertThrows(classOf[UnexpectedAppendOffsetException],
         () => log.appendAsFollower(invalidRecord),
-        () => s"Magic=$magic, compressionType=$compression")
+        () => s"Magic=$magic, compressionType=$compressionType")
     }
   }
 
@@ -2407,23 +2409,23 @@ class UnifiedLogTest {
     val firstOffset = 4L
     val magicVals = Seq(RecordBatch.MAGIC_VALUE_V0, RecordBatch.MAGIC_VALUE_V1, RecordBatch.MAGIC_VALUE_V2)
     val compressionTypes = Seq(CompressionType.NONE, CompressionType.LZ4)
-    for (magic <- magicVals; compression <- compressionTypes) {
+    for (magic <- magicVals; compressionType <- compressionTypes) {
       val batch = TestUtils.records(List(new SimpleRecord("k1".getBytes, "v1".getBytes),
                                          new SimpleRecord("k2".getBytes, "v2".getBytes),
                                          new SimpleRecord("k3".getBytes, "v3".getBytes)),
-                                    magicValue = magic, codec = compression,
+                                    magicValue = magic, codec = Compression.of(compressionType).build(),
                                     baseOffset = firstOffset)
 
       val exception = assertThrows(classOf[UnexpectedAppendOffsetException], () => log.appendAsFollower(records = batch))
-      assertEquals(firstOffset, exception.firstOffset, s"Magic=$magic, compressionType=$compression, UnexpectedAppendOffsetException#firstOffset")
-      assertEquals(firstOffset + 2, exception.lastOffset, s"Magic=$magic, compressionType=$compression, UnexpectedAppendOffsetException#lastOffset")
+      assertEquals(firstOffset, exception.firstOffset, s"Magic=$magic, compressionType=$compressionType, UnexpectedAppendOffsetException#firstOffset")
+      assertEquals(firstOffset + 2, exception.lastOffset, s"Magic=$magic, compressionType=$compressionType, UnexpectedAppendOffsetException#lastOffset")
     }
   }
 
   @Test
   def testAppendWithNoTimestamp(): Unit = {
     val log = createLog(logDir, new LogConfig(new Properties))
-    log.appendAsLeader(MemoryRecords.withRecords(CompressionType.NONE,
+    log.appendAsLeader(MemoryRecords.withRecords(Compression.NONE,
       new SimpleRecord(RecordBatch.NO_TIMESTAMP, "key".getBytes, "value".getBytes)), leaderEpoch = 0)
   }
 
@@ -2588,9 +2590,9 @@ class UnifiedLogTest {
     // When we split the overflowed segment, only one new segment will be created.
 
     val overflowOffset = Int.MaxValue + 1L
-    val batch1 = MemoryRecords.withRecords(overflowOffset, CompressionType.NONE, 0,
+    val batch1 = MemoryRecords.withRecords(overflowOffset, Compression.NONE, 0,
       new SimpleRecord("a".getBytes))
-    val batch2 = MemoryRecords.withRecords(overflowOffset + 1, CompressionType.NONE, 0,
+    val batch2 = MemoryRecords.withRecords(overflowOffset + 1, Compression.NONE, 0,
       new SimpleRecord("b".getBytes))
 
     testDegenerateSplitSegmentWithOverflow(segmentBaseOffset = 0L, List(batch1, batch2))
@@ -2602,7 +2604,7 @@ class UnifiedLogTest {
     // the first offset of the batch is valid, but the last overflows.
 
     val firstBatchBaseOffset = Int.MaxValue - 1
-    val records = MemoryRecords.withRecords(firstBatchBaseOffset, CompressionType.NONE, 0,
+    val records = MemoryRecords.withRecords(firstBatchBaseOffset, Compression.NONE, 0,
       new SimpleRecord("a".getBytes),
       new SimpleRecord("b".getBytes),
       new SimpleRecord("c".getBytes))
@@ -2869,7 +2871,7 @@ class UnifiedLogTest {
     //When appending messages as a leader (i.e. assignOffsets = true)
     for (record <- records)
       log.appendAsLeader(
-        MemoryRecords.withRecords(CompressionType.NONE, record),
+        MemoryRecords.withRecords(Compression.NONE, record),
         leaderEpoch = epoch
       )
 
@@ -2887,7 +2889,7 @@ class UnifiedLogTest {
 
     //Given each message has an offset & epoch, as msgs from leader would
     def recordsForEpoch(i: Int): MemoryRecords = {
-      val recs = MemoryRecords.withRecords(messageIds(i), CompressionType.NONE, records(i))
+      val recs = MemoryRecords.withRecords(messageIds(i), Compression.NONE, records(i))
       recs.batches.forEach{record =>
         record.setPartitionLeaderEpoch(42)
         record.setLastOffset(i)
@@ -3008,7 +3010,7 @@ class UnifiedLogTest {
     val logConfig = LogTestUtils.createLogConfig(segmentBytes = 1024 * 1024 * 5)
     val log = createLog(logDir, logConfig)
 
-    val records = MemoryRecords.withRecords(CompressionType.NONE,
+    val records = MemoryRecords.withRecords(Compression.NONE,
       new SimpleRecord("foo".getBytes),
       new SimpleRecord("bar".getBytes),
       new SimpleRecord("baz".getBytes))
@@ -3027,7 +3029,7 @@ class UnifiedLogTest {
     var seq = 0
 
     // add some transactional records
-    val records = MemoryRecords.withTransactionalRecords(CompressionType.NONE, pid, epoch, seq,
+    val records = MemoryRecords.withTransactionalRecords(Compression.NONE, pid, epoch, seq,
       new SimpleRecord("foo".getBytes),
       new SimpleRecord("bar".getBytes),
       new SimpleRecord("baz".getBytes))
@@ -3037,7 +3039,7 @@ class UnifiedLogTest {
 
     // add more transactional records
     seq += 3
-    log.appendAsLeader(MemoryRecords.withTransactionalRecords(CompressionType.NONE, pid, epoch, seq,
+    log.appendAsLeader(MemoryRecords.withTransactionalRecords(Compression.NONE, pid, epoch, seq,
       new SimpleRecord("blah".getBytes)), leaderEpoch = 0)
 
     // LSO should not have changed
@@ -3406,7 +3408,7 @@ class UnifiedLogTest {
     val log = createLog(logDir, logConfig)
 
     // append a few records
-    appendAsFollower(log, MemoryRecords.withRecords(CompressionType.NONE,
+    appendAsFollower(log, MemoryRecords.withRecords(Compression.NONE,
       new SimpleRecord("a".getBytes),
       new SimpleRecord("b".getBytes),
       new SimpleRecord("c".getBytes)), 5)
@@ -3436,20 +3438,20 @@ class UnifiedLogTest {
     val seq2 = 0
 
     // add some transactional records
-    val firstAppendInfo = log.appendAsLeader(MemoryRecords.withTransactionalRecords(CompressionType.NONE, pid1, epoch, seq1,
+    val firstAppendInfo = log.appendAsLeader(MemoryRecords.withTransactionalRecords(Compression.NONE, pid1, epoch, seq1,
       new SimpleRecord("a".getBytes),
       new SimpleRecord("b".getBytes),
       new SimpleRecord("c".getBytes)), leaderEpoch = 0)
     assertEquals(Some(firstAppendInfo.firstOffset), log.firstUnstableOffset)
 
     // mix in some non-transactional data
-    log.appendAsLeader(MemoryRecords.withRecords(CompressionType.NONE,
+    log.appendAsLeader(MemoryRecords.withRecords(Compression.NONE,
       new SimpleRecord("g".getBytes),
       new SimpleRecord("h".getBytes),
       new SimpleRecord("i".getBytes)), leaderEpoch = 0)
 
     // append data from a second transactional producer
-    val secondAppendInfo = log.appendAsLeader(MemoryRecords.withTransactionalRecords(CompressionType.NONE, pid2, epoch, seq2,
+    val secondAppendInfo = log.appendAsLeader(MemoryRecords.withTransactionalRecords(Compression.NONE, pid2, epoch, seq2,
       new SimpleRecord("d".getBytes),
       new SimpleRecord("e".getBytes),
       new SimpleRecord("f".getBytes)), leaderEpoch = 0)
@@ -3478,7 +3480,7 @@ class UnifiedLogTest {
     val epoch = 5.toShort
     var seq = 0
 
-    val records = MemoryRecords.withTransactionalRecords(CompressionType.NONE, pid, epoch, seq,
+    val records = MemoryRecords.withTransactionalRecords(Compression.NONE, pid, epoch, seq,
       new SimpleRecord("a".getBytes),
       new SimpleRecord("b".getBytes),
       new SimpleRecord("c".getBytes))
@@ -3491,7 +3493,7 @@ class UnifiedLogTest {
 
     // this write should spill to the second segment
     seq = 3
-    log.appendAsLeader(MemoryRecords.withTransactionalRecords(CompressionType.NONE, pid, epoch, seq,
+    log.appendAsLeader(MemoryRecords.withTransactionalRecords(Compression.NONE, pid, epoch, seq,
       new SimpleRecord("d".getBytes),
       new SimpleRecord("e".getBytes),
       new SimpleRecord("f".getBytes)), leaderEpoch = 0)
@@ -3841,7 +3843,7 @@ class UnifiedLogTest {
     assertFalse(log.verificationGuard(producerId).verify(VerificationGuard.SENTINEL))
 
     val idempotentRecords = MemoryRecords.withIdempotentRecords(
-      CompressionType.NONE,
+      Compression.NONE,
       producerId,
       producerEpoch,
       sequence,
@@ -3854,7 +3856,7 @@ class UnifiedLogTest {
       sequence = sequence + 2
 
     val transactionalRecords = MemoryRecords.withTransactionalRecords(
-      CompressionType.NONE,
+      Compression.NONE,
       producerId,
       producerEpoch,
       sequence,
@@ -3939,7 +3941,7 @@ class UnifiedLogTest {
     producerStateManagerConfig.setTransactionVerificationEnabled(false)
 
     val transactionalRecords = MemoryRecords.withTransactionalRecords(
-      CompressionType.NONE,
+      Compression.NONE,
       producerId,
       producerEpoch,
       0,
@@ -3965,7 +3967,7 @@ class UnifiedLogTest {
     producerStateManagerConfig.setTransactionVerificationEnabled(true)
 
     val transactionalRecords = MemoryRecords.withTransactionalRecords(
-      CompressionType.NONE,
+      Compression.NONE,
       producerId,
       producerEpoch,
       sequence,
@@ -3997,7 +3999,7 @@ class UnifiedLogTest {
     assertEquals(VerificationGuard.SENTINEL, log.verificationGuard(producerId))
 
     val transactionalRecords = MemoryRecords.withTransactionalRecords(
-      CompressionType.NONE,
+      Compression.NONE,
       producerId,
       producerEpoch,
       sequence,
@@ -4214,7 +4216,7 @@ class UnifiedLogTest {
                                           leaderEpoch: Int = 0): (Long, Int) => Unit = {
     var sequence = 0
     (offset: Long, numRecords: Int) => {
-      val builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, CompressionType.NONE, TimestampType.CREATE_TIME,
+      val builder = MemoryRecords.builder(buffer, RecordBatch.CURRENT_MAGIC_VALUE, Compression.NONE, TimestampType.CREATE_TIME,
         offset, mockTime.milliseconds(), producerId, producerEpoch, sequence, true, leaderEpoch)
       for (seq <- sequence until sequence + numRecords) {
         val record = new SimpleRecord(s"$seq".getBytes)
@@ -4238,7 +4240,7 @@ class UnifiedLogTest {
   }
 
   private def appendNonTransactionalToBuffer(buffer: ByteBuffer, offset: Long, numRecords: Int): Unit = {
-    val builder = MemoryRecords.builder(buffer, CompressionType.NONE, TimestampType.CREATE_TIME, offset)
+    val builder = MemoryRecords.builder(buffer, Compression.NONE, TimestampType.CREATE_TIME, offset)
     (0 until numRecords).foreach { seq =>
       builder.append(new SimpleRecord(s"$seq".getBytes))
     }
diff --git a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala
index fcd21acd83b..269ebf8a369 100644
--- a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala
+++ b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala
@@ -161,13 +161,14 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness {
     assertEquals(IBP_3_0_IV1, log.config.messageFormatVersion)
     assertEquals(RecordVersion.V2, log.config.recordVersion)
 
-    val compressionType = CompressionType.LZ4.name
+    val compressionType = CompressionType.LZ4
     logProps.put(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG, "0.11.0")
     // set compression type so that we can detect when the config change has propagated
-    logProps.put(TopicConfig.COMPRESSION_TYPE_CONFIG, compressionType)
+    logProps.put(TopicConfig.COMPRESSION_TYPE_CONFIG, compressionType.name)
     adminZkClient.changeTopicConfig(tp.topic, logProps)
     TestUtils.waitUntilTrue(() =>
-      server.logManager.getLog(tp).get.config.compressionType == compressionType,
+      server.logManager.getLog(tp).get.config.compression.isPresent &&
+      server.logManager.getLog(tp).get.config.compression.get.`type` == compressionType,
       "Topic config change propagation failed")
     assertEquals(IBP_3_0_IV1, log.config.messageFormatVersion)
     assertEquals(RecordVersion.V2, log.config.recordVersion)
diff --git a/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala b/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala
index a018b5d6af4..c619acde664 100755
--- a/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala
+++ b/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala
@@ -21,15 +21,15 @@ import java.io.{DataInputStream, DataOutputStream}
 import java.net.Socket
 import java.nio.ByteBuffer
 import java.util.Collections
-
 import kafka.integration.KafkaServerTestHarness
 import kafka.network.SocketServer
 import kafka.utils._
+import org.apache.kafka.common.compress.Compression
 import org.apache.kafka.common.message.ProduceRequestData
 import org.apache.kafka.common.network.ListenerName
 import org.apache.kafka.common.protocol.types.Type
 import org.apache.kafka.common.protocol.{ApiKeys, Errors}
-import org.apache.kafka.common.record.{CompressionType, MemoryRecords, SimpleRecord}
+import org.apache.kafka.common.record.{MemoryRecords, SimpleRecord}
 import org.apache.kafka.common.requests.{ProduceResponse, ResponseHeader}
 import org.apache.kafka.common.security.auth.SecurityProtocol
 import org.apache.kafka.common.utils.ByteUtils
@@ -135,7 +135,7 @@ class EdgeCaseRequestTest extends KafkaServerTestHarness {
             .setName(topicPartition.topic()).setPartitionData(Collections.singletonList(
             new ProduceRequestData.PartitionProduceData()
               .setIndex(topicPartition.partition())
-              .setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("message".getBytes))))))
+              .setRecords(MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("message".getBytes))))))
             .iterator))
         .setAcks(1.toShort)
         .setTimeoutMs(10000)
diff --git a/core/src/test/scala/unit/kafka/server/FetchSessionTest.scala b/core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
index c5f3b269c97..7e5aca381ae 100755
--- a/core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
+++ b/core/src/test/scala/unit/kafka/server/FetchSessionTest.scala
@@ -16,10 +16,10 @@
 */
 package kafka.server
 
+import org.apache.kafka.common.compress.Compression
 import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid}
 import org.apache.kafka.common.message.FetchResponseData
 import org.apache.kafka.common.protocol.{ApiKeys, Errors}
-import org.apache.kafka.common.record.CompressionType
 import org.apache.kafka.common.record.MemoryRecords
 import org.apache.kafka.common.record.SimpleRecord
 import org.apache.kafka.common.requests.FetchMetadata.{FINAL_EPOCH, INVALID_SESSION_ID}
@@ -1854,7 +1854,7 @@ class FetchSessionTest {
       .setHighWatermark(60)
       .setLastStableOffset(50)
       .setLogStartOffset(0)
-      .setRecords(MemoryRecords.withRecords(CompressionType.NONE,
+      .setRecords(MemoryRecords.withRecords(Compression.NONE,
         new SimpleRecord(100, null))))
     respData3.put(tp3, new FetchResponseData.PartitionData()
       .setPartitionIndex(tp3.topicPartition.partition)
diff --git a/core/src/test/scala/unit/kafka/server/FetcherThreadTestUtils.scala b/core/src/test/scala/unit/kafka/server/FetcherThreadTestUtils.scala
index b6d53f6dca3..d22808ecf5c 100644
--- a/core/src/test/scala/unit/kafka/server/FetcherThreadTestUtils.scala
+++ b/core/src/test/scala/unit/kafka/server/FetcherThreadTestUtils.scala
@@ -20,13 +20,14 @@ package kafka.server
 import kafka.cluster.BrokerEndPoint
 import kafka.server.InitialFetchState
 import org.apache.kafka.common.Uuid
-import org.apache.kafka.common.record.{CompressionType, MemoryRecords, RecordBatch, SimpleRecord}
+import org.apache.kafka.common.compress.Compression
+import org.apache.kafka.common.record.{MemoryRecords, RecordBatch, SimpleRecord}
 
 import scala.jdk.CollectionConverters._
 
 object FetcherThreadTestUtils {
   def mkBatch(baseOffset: Long, leaderEpoch: Int, records: SimpleRecord*): RecordBatch = {
-    MemoryRecords.withRecords(baseOffset, CompressionType.NONE, leaderEpoch, records: _*)
+    MemoryRecords.withRecords(baseOffset, Compression.NONE, leaderEpoch, records: _*)
       .batches.asScala.head
   }
 
diff --git a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala
index 6f534f34535..ce7acfe821f 100644
--- a/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala
+++ b/core/src/test/scala/unit/kafka/server/KafkaApisTest.scala
@@ -31,6 +31,7 @@ import org.apache.kafka.clients.admin.AlterConfigOp.OpType
 import org.apache.kafka.clients.admin.{AlterConfigOp, ConfigEntry}
 import org.apache.kafka.common._
 import org.apache.kafka.common.acl.AclOperation
+import org.apache.kafka.common.compress.Compression
 import org.apache.kafka.common.config.ConfigResource
 import org.apache.kafka.common.config.ConfigResource.Type.{BROKER, BROKER_LOGGER}
 import org.apache.kafka.common.errors.{ClusterAuthorizationException, UnsupportedVersionException}
@@ -2474,7 +2475,7 @@ class KafkaApisTest extends Logging {
             .setName(tp.topic).setPartitionData(Collections.singletonList(
             new ProduceRequestData.PartitionProduceData()
               .setIndex(tp.partition)
-              .setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("test".getBytes))))))
+              .setRecords(MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("test".getBytes))))))
             .iterator))
         .setAcks(1.toShort)
         .setTimeoutMs(5000))
@@ -2536,7 +2537,7 @@ class KafkaApisTest extends Logging {
             .setName(tp.topic).setPartitionData(Collections.singletonList(
             new ProduceRequestData.PartitionProduceData()
               .setIndex(tp.partition)
-              .setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("test".getBytes))))))
+              .setRecords(MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("test".getBytes))))))
             .iterator))
         .setAcks(1.toShort)
         .setTimeoutMs(5000))
@@ -2601,7 +2602,7 @@ class KafkaApisTest extends Logging {
             .setName(tp.topic).setPartitionData(Collections.singletonList(
             new ProduceRequestData.PartitionProduceData()
               .setIndex(tp.partition)
-              .setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("test".getBytes))))))
+              .setRecords(MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("test".getBytes))))))
             .iterator))
         .setAcks(1.toShort)
         .setTimeoutMs(5000))
@@ -2665,7 +2666,7 @@ class KafkaApisTest extends Logging {
             .setName(tp.topic).setPartitionData(Collections.singletonList(
             new ProduceRequestData.PartitionProduceData()
               .setIndex(tp.partition)
-              .setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("test".getBytes))))))
+              .setRecords(MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("test".getBytes))))))
             .iterator))
         .setAcks(1.toShort)
         .setTimeoutMs(5000))
@@ -2728,7 +2729,7 @@ class KafkaApisTest extends Logging {
             .setName(tp.topic).setPartitionData(Collections.singletonList(
             new ProduceRequestData.PartitionProduceData()
               .setIndex(tp.partition)
-              .setRecords(MemoryRecords.withTransactionalRecords(CompressionType.NONE, 0, 0, 0, new SimpleRecord("test".getBytes))))))
+              .setRecords(MemoryRecords.withTransactionalRecords(Compression.NONE, 0, 0, 0, new SimpleRecord("test".getBytes))))))
             .iterator))
         .setAcks(1.toShort)
         .setTransactionalId(transactionalId)
@@ -4203,7 +4204,7 @@ class KafkaApisTest extends Logging {
       any[Seq[(TopicIdPartition, FetchPartitionData)] => Unit]()
     )).thenAnswer(invocation => {
       val callback = invocation.getArgument(3).asInstanceOf[Seq[(TopicIdPartition, FetchPartitionData)] => Unit]
-      val records = MemoryRecords.withRecords(CompressionType.NONE,
+      val records = MemoryRecords.withRecords(Compression.NONE,
         new SimpleRecord(timestamp, "foo".getBytes(StandardCharsets.UTF_8)))
       callback(Seq(tidp -> new FetchPartitionData(Errors.NONE, hw, 0, records,
         Optional.empty(), OptionalLong.empty(), Optional.empty(), OptionalInt.empty(), false)))
@@ -5566,7 +5567,7 @@ class KafkaApisTest extends Logging {
     val fetchFromFollower = buildRequest(new FetchRequest.Builder(
       ApiKeys.FETCH.oldestVersion(), ApiKeys.FETCH.latestVersion(), 1, 1, 1000, 0, fetchDataBuilder).build())
 
-    val records = MemoryRecords.withRecords(CompressionType.NONE,
+    val records = MemoryRecords.withRecords(Compression.NONE,
       new SimpleRecord(1000, "foo".getBytes(StandardCharsets.UTF_8)))
     when(replicaManager.fetchMessages(
       any[FetchParams],
@@ -6310,7 +6311,7 @@ class KafkaApisTest extends Logging {
             .setHighWatermark(105)
             .setLastStableOffset(105)
             .setLogStartOffset(0)
-            .setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord(100, raw.getBytes(StandardCharsets.UTF_8))))
+            .setRecords(MemoryRecords.withRecords(Compression.NONE, new SimpleRecord(100, raw.getBytes(StandardCharsets.UTF_8))))
       }.toMap.asJava)
 
       data.foreach{case (tp, _) =>
diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
index e67bcbbc788..8513854be7b 100755
--- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
+++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala
@@ -30,6 +30,7 @@ import org.apache.kafka.common.metrics.Sensor
 import org.apache.kafka.common.network.ListenerName
 import org.apache.kafka.common.record.{CompressionType, Records}
 import org.apache.kafka.common.security.auth.SecurityProtocol
+import org.apache.kafka.common.compress.{GzipCompression, Lz4Compression, ZstdCompression}
 import org.apache.kafka.coordinator.group.ConsumerGroupMigrationPolicy
 import org.apache.kafka.coordinator.group.Group.GroupType
 import org.apache.kafka.coordinator.group.GroupCoordinatorConfig
@@ -691,7 +692,6 @@ class KafkaConfigTest {
   def testDefaultCompressionType(): Unit = {
     val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181)
     val serverConfig = KafkaConfig.fromProps(props)
-
     assertEquals(serverConfig.compressionType, "producer")
   }
 
@@ -700,7 +700,6 @@ class KafkaConfigTest {
     val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181)
     props.setProperty("compression.type", "gzip")
     val serverConfig = KafkaConfig.fromProps(props)
-
     assertEquals(serverConfig.compressionType, "gzip")
   }
 
@@ -711,6 +710,30 @@ class KafkaConfigTest {
     assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props))
   }
 
+  @Test
+  def testInvalidGzipCompressionLevel(): Unit = {
+    val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181)
+    props.setProperty(KafkaConfig.CompressionTypeProp, "gzip")
+    props.setProperty(KafkaConfig.CompressionGzipLevelProp, (GzipCompression.MAX_LEVEL + 1).toString)
+    assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props))
+  }
+
+  @Test
+  def testInvalidLz4CompressionLevel(): Unit = {
+    val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181)
+    props.setProperty(KafkaConfig.CompressionTypeProp, "lz4")
+    props.setProperty(KafkaConfig.CompressionLz4LevelProp, (Lz4Compression.MAX_LEVEL + 1).toString)
+    assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props))
+  }
+
+  @Test
+  def testInvalidZstdCompressionLevel(): Unit = {
+    val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181)
+    props.setProperty(KafkaConfig.CompressionTypeProp, "zstd")
+    props.setProperty(KafkaConfig.CompressionZstdLevelProp, (ZstdCompression.MAX_LEVEL + 1).toString)
+    assertThrows(classOf[ConfigException], () => KafkaConfig.fromProps(props))
+  }
+
   @Test
   def testInvalidInterBrokerSecurityProtocol(): Unit = {
     val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 8181)
@@ -917,6 +940,11 @@ class KafkaConfigTest {
         case MetricConfigs.METRIC_SAMPLE_WINDOW_MS_CONFIG => assertPropertyInvalid(baseProperties, name, "not_a_number", "-1", "0")
         case MetricConfigs.METRIC_REPORTER_CLASSES_CONFIG => // ignore string
         case MetricConfigs.METRIC_RECORDING_LEVEL_CONFIG => // ignore string
+
+        case KafkaConfig.CompressionGzipLevelProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0")
+        case KafkaConfig.CompressionLz4LevelProp => assertPropertyInvalid(baseProperties, name, "not_a_number", "0")
+        case KafkaConfig.CompressionZstdLevelProp => assertPropertyInvalid(baseProperties, name, "not_a_number", ZstdCompression.MAX_LEVEL + 1)
+
         case KafkaConfig.RackProp => // ignore string
         //SSL Configs
         case KafkaSecurityConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG =>
@@ -1078,6 +1106,12 @@ class KafkaConfigTest {
           assertDynamic(kafkaConfigProp, TopicConfig.CLEANUP_POLICY_COMPACT, () => config.logCleanupPolicy)
         case TopicConfig.COMPRESSION_TYPE_CONFIG =>
           assertDynamic(kafkaConfigProp, "lz4", () => config.compressionType)
+        case TopicConfig.COMPRESSION_GZIP_LEVEL_CONFIG =>
+          assertDynamic(kafkaConfigProp, "5", () => config.gzipCompressionLevel)
+        case TopicConfig.COMPRESSION_LZ4_LEVEL_CONFIG =>
+          assertDynamic(kafkaConfigProp, "5", () => config.lz4CompressionLevel)
+        case TopicConfig.COMPRESSION_ZSTD_LEVEL_CONFIG =>
+          assertDynamic(kafkaConfigProp, "5", () => config.zstdCompressionLevel)
         case TopicConfig.SEGMENT_BYTES_CONFIG =>
           assertDynamic(kafkaConfigProp, 10000, () => config.logSegmentBytes)
         case TopicConfig.SEGMENT_MS_CONFIG =>
diff --git a/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala b/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala
index ee8d3c6b6b6..fea41b76112 100644
--- a/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala
@@ -21,6 +21,7 @@ import java.nio.ByteBuffer
 import java.util.{Collections, Properties}
 import kafka.utils.TestUtils
 import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.compress.Compression
 import org.apache.kafka.common.config.TopicConfig
 import org.apache.kafka.common.message.ProduceRequestData
 import org.apache.kafka.common.protocol.Errors
@@ -74,10 +75,10 @@ class ProduceRequestTest extends BaseRequestTest {
       assertTrue(partitionProduceResponse.recordErrors.isEmpty)
     }
 
-    sendAndCheck(MemoryRecords.withRecords(CompressionType.NONE,
+    sendAndCheck(MemoryRecords.withRecords(Compression.NONE,
       new SimpleRecord(System.currentTimeMillis(), "key".getBytes, "value".getBytes)), 0)
 
-    sendAndCheck(MemoryRecords.withRecords(CompressionType.GZIP,
+    sendAndCheck(MemoryRecords.withRecords(Compression.gzip().build(),
       new SimpleRecord(System.currentTimeMillis(), "key1".getBytes, "value1".getBytes),
       new SimpleRecord(System.currentTimeMillis(), "key2".getBytes, "value2".getBytes)), 1)
   }
@@ -92,7 +93,7 @@ class ProduceRequestTest extends BaseRequestTest {
     val partitionToLeader = TestUtils.createTopic(zkClient, topic, 1, 1, servers, topicConfig)
     val leader = partitionToLeader(partition)
 
-    def createRecords(magicValue: Byte, timestamp: Long, codec: CompressionType): MemoryRecords = {
+    def createRecords(magicValue: Byte, timestamp: Long, codec: Compression): MemoryRecords = {
       val buf = ByteBuffer.allocate(512)
       val builder = MemoryRecords.builder(buf, magicValue, codec, TimestampType.CREATE_TIME, 0L)
       builder.appendWithOffset(0, timestamp, null, "hello".getBytes)
@@ -101,7 +102,7 @@ class ProduceRequestTest extends BaseRequestTest {
       builder.build()
     }
 
-    val records = createRecords(RecordBatch.MAGIC_VALUE_V2, recordTimestamp, CompressionType.GZIP)
+    val records = createRecords(RecordBatch.MAGIC_VALUE_V2, recordTimestamp, Compression.gzip().build())
     val topicPartition = new TopicPartition("topic", partition)
     val produceResponse = sendProduceRequest(leader, ProduceRequest.forCurrentMagic(new ProduceRequestData()
       .setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList(
@@ -143,7 +144,7 @@ class ProduceRequestTest extends BaseRequestTest {
     val nonReplicaId =  nonReplicaOpt.get.config.brokerId
 
     // Send the produce request to the non-replica
-    val records = MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("key".getBytes, "value".getBytes))
+    val records = MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("key".getBytes, "value".getBytes))
     val topicPartition = new TopicPartition("topic", partition)
     val produceRequest = ProduceRequest.forCurrentMagic(new ProduceRequestData()
       .setTopicData(new ProduceRequestData.TopicProduceDataCollection(Collections.singletonList(
@@ -177,7 +178,7 @@ class ProduceRequestTest extends BaseRequestTest {
   def testCorruptLz4ProduceRequest(quorum: String): Unit = {
     val (partition, leader) = createTopicAndFindPartitionWithLeader("topic")
     val timestamp = 1000000
-    val memoryRecords = MemoryRecords.withRecords(CompressionType.LZ4,
+    val memoryRecords = MemoryRecords.withRecords(Compression.lz4().build(),
       new SimpleRecord(timestamp, "key".getBytes, "value".getBytes))
     // Change the lz4 checksum value (not the kafka record crc) so that it doesn't match the contents
     val lz4ChecksumOffset = 6
@@ -218,7 +219,7 @@ class ProduceRequestTest extends BaseRequestTest {
     topicConfig.setProperty(TopicConfig.COMPRESSION_TYPE_CONFIG, BrokerCompressionType.ZSTD.name)
     val partitionToLeader = createTopic(topic, topicConfig =  topicConfig)
     val leader = partitionToLeader(partition)
-    val memoryRecords = MemoryRecords.withRecords(CompressionType.ZSTD,
+    val memoryRecords = MemoryRecords.withRecords(Compression.zstd().build(),
       new SimpleRecord(System.currentTimeMillis(), "key".getBytes, "value".getBytes))
     val topicPartition = new TopicPartition("topic", partition)
     val partitionRecords = new ProduceRequestData()
diff --git a/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala
index eb90db6b451..7d2e1997087 100644
--- a/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ReplicaFetcherThreadTest.scala
@@ -24,6 +24,7 @@ import kafka.server.epoch.util.MockBlockingSender
 import kafka.server.metadata.ZkMetadataCache
 import kafka.utils.TestUtils
 import org.apache.kafka.clients.FetchSessionHandler
+import org.apache.kafka.common.compress.Compression
 import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid}
 import org.apache.kafka.common.message.{FetchResponseData, UpdateMetadataRequestData}
 import org.apache.kafka.common.message.OffsetForLeaderEpochRequestData.OffsetForLeaderPartition
@@ -1331,7 +1332,7 @@ class ReplicaFetcherThreadTest {
 
     val tp0 = new TopicPartition("testTopic", 0)
     val tp1 = new TopicPartition("testTopic", 1)
-    val records = MemoryRecords.withRecords(CompressionType.NONE,
+    val records = MemoryRecords.withRecords(Compression.NONE,
       new SimpleRecord(1000, "foo".getBytes(StandardCharsets.UTF_8)))
     val partitionData = new FetchResponseData.PartitionData()
       .setRecords(records)
@@ -1380,7 +1381,7 @@ class ReplicaFetcherThreadTest {
     when(mockBlockingSend.brokerEndPoint()).thenReturn(brokerEndPoint)
 
     val log: UnifiedLog = mock(classOf[UnifiedLog])
-    val records = MemoryRecords.withRecords(CompressionType.NONE,
+    val records = MemoryRecords.withRecords(Compression.NONE,
       new SimpleRecord(1000, "foo".getBytes(StandardCharsets.UTF_8)))
     when(log.maybeUpdateHighWatermark(hw = 0)).thenReturn(None)
 
diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala
index 1006a3e0471..17583c269e7 100644
--- a/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala
@@ -22,9 +22,10 @@ import kafka.cluster.{Partition, PartitionTest}
 import kafka.log.{LogManager, UnifiedLog}
 import kafka.server.QuotaFactory.QuotaManagers
 import kafka.utils._
+import org.apache.kafka.common.compress.Compression
 import org.apache.kafka.common.metrics.Metrics
 import org.apache.kafka.common.protocol.ApiKeys
-import org.apache.kafka.common.record.{CompressionType, MemoryRecords, SimpleRecord}
+import org.apache.kafka.common.record.{MemoryRecords, SimpleRecord}
 import org.apache.kafka.common.requests.FetchRequest
 import org.apache.kafka.common.requests.FetchRequest.PartitionData
 import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid}
@@ -270,7 +271,7 @@ class ReplicaManagerQuotasTest {
       minOneMessage = anyBoolean)).thenReturn(
       new FetchDataInfo(
         new LogOffsetMetadata(0L, 0L, 0),
-        MemoryRecords.withRecords(CompressionType.NONE, record)
+        MemoryRecords.withRecords(Compression.NONE, record)
       ))
 
     //if we ask for len = 0, return 0 messages
diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
index ab258642ceb..5f1c41ba3f1 100644
--- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
@@ -30,7 +30,8 @@ import kafka.utils.TestUtils.waitUntilTrue
 import kafka.utils.{Exit, Pool, TestUtils}
 import kafka.zk.KafkaZkClient
 import org.apache.kafka.clients.FetchSessionHandler
-import org.apache.kafka.common._
+import org.apache.kafka.common.{DirectoryId, IsolationLevel, Node, TopicIdPartition, TopicPartition, Uuid}
+import org.apache.kafka.common.compress.Compression
 import org.apache.kafka.common.config.{AbstractConfig, TopicConfig}
 import org.apache.kafka.common.errors.{InvalidPidMappingException, KafkaStorageException}
 import org.apache.kafka.common.message.LeaderAndIsrRequestData
@@ -229,7 +230,7 @@ class ReplicaManagerTest {
         requiredAcks = 3,
         internalTopicsAllowed = false,
         origin = AppendOrigin.CLIENT,
-        entriesPerPartition = Map(new TopicPartition("test1", 0) -> MemoryRecords.withRecords(CompressionType.NONE,
+        entriesPerPartition = Map(new TopicPartition("test1", 0) -> MemoryRecords.withRecords(Compression.NONE,
           new SimpleRecord("first message".getBytes))),
         responseCallback = callback)
     } finally {
@@ -293,7 +294,7 @@ class ReplicaManagerTest {
         Collections.singletonMap(topic, Uuid.randomUuid()),
         Set(new Node(0, "host1", 0)).asJava).build(), (_, _) => ())
       appendRecords(rm, new TopicPartition(topic, 0),
-        MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("first message".getBytes()), new SimpleRecord("second message".getBytes())))
+        MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("first message".getBytes()), new SimpleRecord("second message".getBytes())))
       logManager.maybeUpdatePreferredLogDir(new TopicPartition(topic, 0), dir2.getAbsolutePath)
 
       partition.createLogIfNotExists(isNew = true, isFutureReplica = true,
@@ -361,7 +362,7 @@ class ReplicaManagerTest {
       rm.getPartitionOrException(new TopicPartition(topic, 0))
         .localLogOrException
 
-      val records = MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("first message".getBytes()))
+      val records = MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("first message".getBytes()))
       val appendResult = appendRecords(rm, new TopicPartition(topic, 0), records).onFire { response =>
         assertEquals(Errors.NOT_LEADER_OR_FOLLOWER, response.error)
       }
@@ -527,7 +528,7 @@ class ReplicaManagerTest {
       // write a few batches as part of a transaction
       val numRecords = 3
       for (sequence <- 0 until numRecords) {
-        val records = MemoryRecords.withIdempotentRecords(CompressionType.NONE, producerId, epoch, sequence,
+        val records = MemoryRecords.withIdempotentRecords(Compression.NONE, producerId, epoch, sequence,
           new SimpleRecord(s"message $sequence".getBytes))
         appendRecords(replicaManager, new TopicPartition(topic, 0), records).onFire { response =>
           assertEquals(Errors.NONE, response.error)
@@ -539,7 +540,7 @@ class ReplicaManagerTest {
       // Append a record with an out of range sequence. We should get the OutOfOrderSequence error code with the log
       // start offset set.
       val outOfRangeSequence = numRecords + 10
-      val record = MemoryRecords.withIdempotentRecords(CompressionType.NONE, producerId, epoch, outOfRangeSequence,
+      val record = MemoryRecords.withIdempotentRecords(Compression.NONE, producerId, epoch, outOfRangeSequence,
         new SimpleRecord(s"message: $outOfRangeSequence".getBytes))
       appendRecords(replicaManager, new TopicPartition(topic, 0), record).onFire { response =>
         assertEquals(Errors.OUT_OF_ORDER_SEQUENCE_NUMBER, response.error)
@@ -592,7 +593,7 @@ class ReplicaManagerTest {
 
       def appendRecord(pid: Long, sequence: Int, partition: Int): Unit = {
         val epoch = 42.toShort
-        val records = MemoryRecords.withIdempotentRecords(CompressionType.NONE, pid, epoch, sequence,
+        val records = MemoryRecords.withIdempotentRecords(Compression.NONE, pid, epoch, sequence,
           new SimpleRecord(s"message $sequence".getBytes))
         appendRecords(replicaManager, new TopicPartition(topic, partition), records).onFire { response =>
           assertEquals(Errors.NONE, response.error)
@@ -674,7 +675,7 @@ class ReplicaManagerTest {
       val producerId = 234L
       val epoch = 5.toShort
       val sequence = 9
-      val records = MemoryRecords.withTransactionalRecords(CompressionType.NONE, producerId, epoch, sequence,
+      val records = MemoryRecords.withTransactionalRecords(Compression.NONE, producerId, epoch, sequence,
         new SimpleRecord(time.milliseconds(), s"message $sequence".getBytes))
       handleProduceAppend(replicaManager, new TopicPartition(topic, 0), records, transactionalId = transactionalId).onFire { response =>
         assertEquals(Errors.NONE, response.error)
@@ -738,7 +739,7 @@ class ReplicaManagerTest {
       // write a few batches as part of a transaction
       val numRecords = 3
       for (sequence <- 0 until numRecords) {
-        val records = MemoryRecords.withTransactionalRecords(CompressionType.NONE, producerId, epoch, sequence,
+        val records = MemoryRecords.withTransactionalRecords(Compression.NONE, producerId, epoch, sequence,
           new SimpleRecord(s"message $sequence".getBytes))
         handleProduceAppend(replicaManager, new TopicPartition(topic, 0), records, transactionalId = transactionalId).onFire { response =>
           assertEquals(Errors.NONE, response.error)
@@ -859,7 +860,7 @@ class ReplicaManagerTest {
       // write a few batches as part of a transaction
       val numRecords = 3
       for (sequence <- 0 until numRecords) {
-        val records = MemoryRecords.withTransactionalRecords(CompressionType.NONE, producerId, epoch, sequence,
+        val records = MemoryRecords.withTransactionalRecords(Compression.NONE, producerId, epoch, sequence,
           new SimpleRecord(s"message $sequence".getBytes))
         handleProduceAppend(replicaManager, new TopicPartition(topic, 0), records, transactionalId = transactionalId).onFire { response =>
           assertEquals(Errors.NONE, response.error)
@@ -1748,7 +1749,7 @@ class ReplicaManagerTest {
 
       val simpleRecords = Seq(new SimpleRecord("a".getBytes), new SimpleRecord("b".getBytes))
       val appendResult = appendRecords(replicaManager, tp0,
-        MemoryRecords.withRecords(CompressionType.NONE, simpleRecords.toSeq: _*), AppendOrigin.CLIENT)
+        MemoryRecords.withRecords(Compression.NONE, simpleRecords.toSeq: _*), AppendOrigin.CLIENT)
 
       // Increment the hw in the leader by fetching from the last offset
       val fetchOffset = simpleRecords.size
@@ -2182,17 +2183,17 @@ class ReplicaManagerTest {
         (_, _) => ())
 
       // If we supply no transactional ID and idempotent records, we do not verify.
-      val idempotentRecords = MemoryRecords.withIdempotentRecords(CompressionType.NONE, producerId, producerEpoch, sequence,
+      val idempotentRecords = MemoryRecords.withIdempotentRecords(Compression.NONE, producerId, producerEpoch, sequence,
         new SimpleRecord("message".getBytes))
       handleProduceAppend(replicaManager, tp0, idempotentRecords, transactionalId = null)
       verify(addPartitionsToTxnManager, times(0)).verifyTransaction(any(), any(), any(), any(), any[AddPartitionsToTxnManager.AppendCallback](), any())
       assertEquals(VerificationGuard.SENTINEL, getVerificationGuard(replicaManager, tp0, producerId))
 
       // If we supply a transactional ID and some transactional and some idempotent records, we should only verify the topic partition with transactional records.
-      val transactionalRecords = MemoryRecords.withTransactionalRecords(CompressionType.NONE, producerId, producerEpoch, sequence + 1,
+      val transactionalRecords = MemoryRecords.withTransactionalRecords(Compression.NONE, producerId, producerEpoch, sequence + 1,
         new SimpleRecord("message".getBytes))
 
-      val idempotentRecords2 = MemoryRecords.withIdempotentRecords(CompressionType.NONE, producerId, producerEpoch, sequence,
+      val idempotentRecords2 = MemoryRecords.withIdempotentRecords(Compression.NONE, producerId, producerEpoch, sequence,
         new SimpleRecord("message".getBytes))
       handleProduceAppendToMultipleTopics(replicaManager, Map(tp0 -> transactionalRecords, tp1 -> idempotentRecords2), transactionalId)
       verify(addPartitionsToTxnManager, times(1)).verifyTransaction(
@@ -2226,7 +2227,7 @@ class ReplicaManagerTest {
         (_, _) => ())
 
       // Append some transactional records.
-      val transactionalRecords = MemoryRecords.withTransactionalRecords(CompressionType.NONE, producerId, producerEpoch, sequence,
+      val transactionalRecords = MemoryRecords.withTransactionalRecords(Compression.NONE, producerId, producerEpoch, sequence,
         new SimpleRecord("message".getBytes))
 
       // We should add these partitions to the manager to verify.
@@ -2286,7 +2287,7 @@ class ReplicaManagerTest {
         (_, _) => ())
 
       // Start with sequence 6
-      val transactionalRecords = MemoryRecords.withTransactionalRecords(CompressionType.NONE, producerId, producerEpoch, sequence,
+      val transactionalRecords = MemoryRecords.withTransactionalRecords(Compression.NONE, producerId, producerEpoch, sequence,
         new SimpleRecord("message".getBytes))
 
       // We should add these partitions to the manager to verify.
@@ -2310,7 +2311,7 @@ class ReplicaManagerTest {
       assertEquals(verificationGuard, getVerificationGuard(replicaManager, tp0, producerId))
 
       // Try to append a higher sequence (7) after the first one failed with a retriable error.
-      val transactionalRecords2 = MemoryRecords.withTransactionalRecords(CompressionType.NONE, producerId, producerEpoch, sequence + 1,
+      val transactionalRecords2 = MemoryRecords.withTransactionalRecords(Compression.NONE, producerId, producerEpoch, sequence + 1,
         new SimpleRecord("message".getBytes))
 
       val result2 = handleProduceAppend(replicaManager, tp0, transactionalRecords2, transactionalId = transactionalId)
@@ -2354,7 +2355,7 @@ class ReplicaManagerTest {
         makeLeaderAndIsrRequest(topicIds(tp1.topic), tp1, Seq(0, 1), LeaderAndIsr(0, List(0, 1))),
         (_, _) => ())
 
-      val transactionalRecords = MemoryRecords.withTransactionalRecords(CompressionType.NONE, producerId, producerEpoch, sequence,
+      val transactionalRecords = MemoryRecords.withTransactionalRecords(Compression.NONE, producerId, producerEpoch, sequence,
         new SimpleRecord(s"message $sequence".getBytes))
 
       handleProduceAppendToMultipleTopics(replicaManager, Map(tp0 -> transactionalRecords, tp1 -> transactionalRecords), transactionalId).onFire { responses =>
@@ -2391,9 +2392,9 @@ class ReplicaManagerTest {
 
       // Append some transactional records with different producer IDs
       val transactionalRecords = mutable.Map[TopicPartition, MemoryRecords]()
-      transactionalRecords.put(tp0, MemoryRecords.withTransactionalRecords(CompressionType.NONE, producerId, producerEpoch, sequence,
+      transactionalRecords.put(tp0, MemoryRecords.withTransactionalRecords(Compression.NONE, producerId, producerEpoch, sequence,
         new SimpleRecord(s"message $sequence".getBytes)))
-      transactionalRecords.put(tp1, MemoryRecords.withTransactionalRecords(CompressionType.NONE, producerId + 1, producerEpoch, sequence,
+      transactionalRecords.put(tp1, MemoryRecords.withTransactionalRecords(Compression.NONE, producerId + 1, producerEpoch, sequence,
         new SimpleRecord(s"message $sequence".getBytes)))
 
       assertThrows(classOf[InvalidPidMappingException],
@@ -2416,7 +2417,7 @@ class ReplicaManagerTest {
     val replicaManager = setUpReplicaManagerWithMockedAddPartitionsToTxnManager(addPartitionsToTxnManager, List(tp0))
     try {
       // Append some transactional records.
-      val transactionalRecords = MemoryRecords.withTransactionalRecords(CompressionType.NONE, producerId, producerEpoch, sequence,
+      val transactionalRecords = MemoryRecords.withTransactionalRecords(Compression.NONE, producerId, producerEpoch, sequence,
         new SimpleRecord("message".getBytes))
 
       // We should not add these partitions to the manager to verify, but instead throw an error.
@@ -2449,7 +2450,7 @@ class ReplicaManagerTest {
       val becomeLeaderRequest = makeLeaderAndIsrRequest(topicIds(tp.topic), tp, Seq(0, 1), LeaderAndIsr(0, List(0, 1)))
       replicaManager.becomeLeaderOrFollower(1, becomeLeaderRequest, (_, _) => ())
 
-      val transactionalRecords = MemoryRecords.withTransactionalRecords(CompressionType.NONE, producerId, producerEpoch, sequence,
+      val transactionalRecords = MemoryRecords.withTransactionalRecords(Compression.NONE, producerId, producerEpoch, sequence,
         new SimpleRecord(s"message $sequence".getBytes))
       handleProduceAppend(replicaManager, tp, transactionalRecords, transactionalId = transactionalId).onFire { response =>
         assertEquals(Errors.NONE, response.error)
@@ -2467,7 +2468,7 @@ class ReplicaManagerTest {
       TestUtils.waitUntilTrue(() => config.transactionPartitionVerificationEnable == true, "Config did not dynamically update.")
 
       // Try to append more records. We don't need to send a request since the transaction is already ongoing.
-      val moreTransactionalRecords = MemoryRecords.withTransactionalRecords(CompressionType.NONE, producerId, producerEpoch, sequence + 1,
+      val moreTransactionalRecords = MemoryRecords.withTransactionalRecords(Compression.NONE, producerId, producerEpoch, sequence + 1,
         new SimpleRecord("message".getBytes))
 
       handleProduceAppend(replicaManager, tp, moreTransactionalRecords, transactionalId = transactionalId)
@@ -2494,7 +2495,7 @@ class ReplicaManagerTest {
         (_, _) => ())
 
       // Append some transactional records.
-      val transactionalRecords = MemoryRecords.withTransactionalRecords(CompressionType.NONE, producerId, producerEpoch, sequence,
+      val transactionalRecords = MemoryRecords.withTransactionalRecords(Compression.NONE, producerId, producerEpoch, sequence,
         new SimpleRecord("message".getBytes))
 
       // We should add these partitions to the manager to verify.
@@ -2558,7 +2559,7 @@ class ReplicaManagerTest {
         makeLeaderAndIsrRequest(topicIds(tp0.topic), tp0, Seq(0, 1), LeaderAndIsr(1, List(0, 1))),
         (_, _) => ())
 
-      val transactionalRecords = MemoryRecords.withTransactionalRecords(CompressionType.NONE, producerId, producerEpoch, sequence,
+      val transactionalRecords = MemoryRecords.withTransactionalRecords(Compression.NONE, producerId, producerEpoch, sequence,
         new SimpleRecord("message".getBytes))
 
       // Start verification and return the coordinator related errors.
@@ -2786,7 +2787,7 @@ class ReplicaManagerTest {
     }
 
     val records = MemoryRecords.withRecords(
-      CompressionType.NONE,
+      Compression.NONE,
       IntStream
         .range(0, numOfRecords)
         .mapToObj(i => new SimpleRecord(i.toString.getBytes))
@@ -5691,7 +5692,7 @@ class ReplicaManagerTest {
         replicaManager.getPartition(topicPartition) match {
           case HostedPartition.Online(partition) =>
             partition.appendRecordsToFollowerOrFutureReplica(
-              records = MemoryRecords.withRecords(CompressionType.NONE, 0,
+              records = MemoryRecords.withRecords(Compression.NONE, 0,
                 new SimpleRecord("first message".getBytes)),
               isFuture = false
             )
diff --git a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala
index 4b46a8698fe..7314d9ed612 100644
--- a/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala
+++ b/core/src/test/scala/unit/kafka/server/RequestQuotaTest.scala
@@ -19,6 +19,7 @@ import kafka.security.authorizer.AclAuthorizer
 import kafka.utils.TestUtils
 import org.apache.kafka.common._
 import org.apache.kafka.common.acl._
+import org.apache.kafka.common.compress.Compression
 import org.apache.kafka.common.config.{ConfigResource, TopicConfig}
 import org.apache.kafka.common.internals.Topic
 import org.apache.kafka.common.message.CreatePartitionsRequestData.CreatePartitionsTopic
@@ -266,7 +267,7 @@ class RequestQuotaTest extends BaseRequestTest {
                 .setName(tp.topic()).setPartitionData(Collections.singletonList(
                 new ProduceRequestData.PartitionProduceData()
                   .setIndex(tp.partition())
-                  .setRecords(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord("test".getBytes))))))
+                  .setRecords(MemoryRecords.withRecords(Compression.NONE, new SimpleRecord("test".getBytes))))))
                 .iterator))
             .setAcks(1.toShort)
             .setTimeoutMs(5000))
diff --git a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala
index d433afd6cb4..de7736973b2 100644
--- a/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala
+++ b/core/src/test/scala/unit/kafka/tools/DumpLogSegmentsTest.scala
@@ -32,10 +32,11 @@ import kafka.utils.TestUtils
 import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.{Assignment, Subscription}
 import org.apache.kafka.clients.consumer.internals.ConsumerProtocol
 import org.apache.kafka.common.{TopicPartition, Uuid}
+import org.apache.kafka.common.compress.Compression
 import org.apache.kafka.common.config.TopicConfig
 import org.apache.kafka.common.metadata.{PartitionChangeRecord, RegisterBrokerRecord, TopicRecord}
 import org.apache.kafka.common.protocol.{ByteBufferAccessor, ObjectSerializationCache}
-import org.apache.kafka.common.record.{CompressionType, ControlRecordType, EndTransactionMarker, MemoryRecords, Record, RecordVersion, SimpleRecord}
+import org.apache.kafka.common.record.{ControlRecordType, EndTransactionMarker, MemoryRecords, Record, RecordVersion, SimpleRecord}
 import org.apache.kafka.common.utils.Utils
 import org.apache.kafka.coordinator.group.{CoordinatorRecord, CoordinatorRecordSerde}
 import org.apache.kafka.coordinator.group.generated.{ConsumerGroupMemberMetadataValue, ConsumerGroupMetadataKey, ConsumerGroupMetadataValue, GroupMetadataKey, GroupMetadataValue}
@@ -106,7 +107,7 @@ class DumpLogSegmentsTest {
     batches += BatchInfo(fourthBatchRecords, hasKeys = false, hasValues = false)
 
     batches.foreach { batchInfo =>
-      log.appendAsLeader(MemoryRecords.withRecords(CompressionType.NONE, 0, batchInfo.records: _*),
+      log.appendAsLeader(MemoryRecords.withRecords(Compression.NONE, 0, batchInfo.records: _*),
         leaderEpoch = 0)
     }
     // Flush, but don't close so that the indexes are not trimmed and contain some zero entries
@@ -121,27 +122,27 @@ class DumpLogSegmentsTest {
 
   @Test
   def testBatchAndRecordMetadataOutput(): Unit = {
-    log.appendAsLeader(MemoryRecords.withRecords(CompressionType.NONE, 0,
+    log.appendAsLeader(MemoryRecords.withRecords(Compression.NONE, 0,
       new SimpleRecord("a".getBytes),
       new SimpleRecord("b".getBytes)
     ), leaderEpoch = 0)
 
-    log.appendAsLeader(MemoryRecords.withRecords(CompressionType.GZIP, 0,
+    log.appendAsLeader(MemoryRecords.withRecords(Compression.gzip().build(), 0,
       new SimpleRecord(time.milliseconds(), "c".getBytes, "1".getBytes),
       new SimpleRecord("d".getBytes)
     ), leaderEpoch = 3)
 
-    log.appendAsLeader(MemoryRecords.withRecords(CompressionType.NONE, 0,
+    log.appendAsLeader(MemoryRecords.withRecords(Compression.NONE, 0,
       new SimpleRecord("e".getBytes, null),
       new SimpleRecord(null, "f".getBytes),
       new SimpleRecord("g".getBytes)
     ), leaderEpoch = 3)
 
-    log.appendAsLeader(MemoryRecords.withIdempotentRecords(CompressionType.NONE, 29342342L, 15.toShort, 234123,
+    log.appendAsLeader(MemoryRecords.withIdempotentRecords(Compression.NONE, 29342342L, 15.toShort, 234123,
       new SimpleRecord("h".getBytes)
     ), leaderEpoch = 3)
 
-    log.appendAsLeader(MemoryRecords.withTransactionalRecords(CompressionType.GZIP, 98323L, 99.toShort, 266,
+    log.appendAsLeader(MemoryRecords.withTransactionalRecords(Compression.gzip().build(), 98323L, 99.toShort, 266,
       new SimpleRecord("i".getBytes),
       new SimpleRecord("j".getBytes)
     ), leaderEpoch = 5)
@@ -270,7 +271,7 @@ class DumpLogSegmentsTest {
       buf.flip()
       new SimpleRecord(null, buf.array)
     }).toArray
-    log.appendAsLeader(MemoryRecords.withRecords(CompressionType.NONE, records:_*), leaderEpoch = 1)
+    log.appendAsLeader(MemoryRecords.withRecords(Compression.NONE, records:_*), leaderEpoch = 1)
     log.flush(false)
 
     var output = runDumpLogSegments(Array("--cluster-metadata-decoder", "--files", logFilePath))
@@ -287,8 +288,8 @@ class DumpLogSegmentsTest {
     val writer = new ByteBufferAccessor(buf)
     writer.writeUnsignedVarint(10000)
     writer.writeUnsignedVarint(10000)
-    log.appendAsLeader(MemoryRecords.withRecords(CompressionType.NONE, new SimpleRecord(null, buf.array)), leaderEpoch = 2)
-    log.appendAsLeader(MemoryRecords.withRecords(CompressionType.NONE, records:_*), leaderEpoch = 2)
+    log.appendAsLeader(MemoryRecords.withRecords(Compression.NONE, new SimpleRecord(null, buf.array)), leaderEpoch = 2)
+    log.appendAsLeader(MemoryRecords.withRecords(Compression.NONE, records:_*), leaderEpoch = 2)
 
     output = runDumpLogSegments(Array("--cluster-metadata-decoder", "--skip-record-metadata", "--files", logFilePath))
     assertTrue(output.contains("TOPIC_RECORD"))
diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
index ab1b3738cd5..8ba2ea00cdd 100755
--- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala
+++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
@@ -35,6 +35,7 @@ import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, Produce
 import org.apache.kafka.clients.{ClientResponse, CommonClientConfigs}
 import org.apache.kafka.common._
 import org.apache.kafka.common.acl.{AccessControlEntry, AccessControlEntryFilter, AclBindingFilter}
+import org.apache.kafka.common.compress.Compression
 import org.apache.kafka.common.config.{ConfigException, ConfigResource}
 import org.apache.kafka.common.errors.{KafkaStorageException, OperationNotAttemptedException, TopicExistsException, UnknownTopicOrPartitionException}
 import org.apache.kafka.common.header.Header
@@ -613,7 +614,7 @@ object TestUtils extends Logging {
    */
   def singletonRecords(value: Array[Byte],
                        key: Array[Byte] = null,
-                       codec: CompressionType = CompressionType.NONE,
+                       codec: Compression = Compression.NONE,
                        timestamp: Long = RecordBatch.NO_TIMESTAMP,
                        magicValue: Byte = RecordBatch.CURRENT_MAGIC_VALUE): MemoryRecords = {
     records(Seq(new SimpleRecord(timestamp, key, value)), magicValue = magicValue, codec = codec)
@@ -621,7 +622,7 @@ object TestUtils extends Logging {
 
   def records(records: Iterable[SimpleRecord],
               magicValue: Byte = RecordBatch.CURRENT_MAGIC_VALUE,
-              codec: CompressionType = CompressionType.NONE,
+              codec: Compression = Compression.NONE,
               producerId: Long = RecordBatch.NO_PRODUCER_ID,
               producerEpoch: Short = RecordBatch.NO_PRODUCER_EPOCH,
               sequence: Int = RecordBatch.NO_SEQUENCE,
diff --git a/gradle/spotbugs-exclude.xml b/gradle/spotbugs-exclude.xml
index e76d9dfc6ee..8fff85846c3 100644
--- a/gradle/spotbugs-exclude.xml
+++ b/gradle/spotbugs-exclude.xml
@@ -122,7 +122,7 @@ For a detailed description of spotbugs bug categories, see https://spotbugs.read
 
     
     , U> implements Aut
         private CoordinatorRuntimeMetrics runtimeMetrics;
         private CoordinatorMetrics coordinatorMetrics;
         private Serializer serializer;
-        private CompressionType compressionType;
+        private Compression compression;
 
         public Builder withLogPrefix(String logPrefix) {
             this.logPrefix = logPrefix;
@@ -176,8 +176,8 @@ public class CoordinatorRuntime, U> implements Aut
             return this;
         }
 
-        public Builder withCompressionType(CompressionType compressionType) {
-            this.compressionType = compressionType;
+        public Builder withCompression(Compression compression) {
+            this.compression = compression;
             return this;
         }
 
@@ -204,8 +204,8 @@ public class CoordinatorRuntime, U> implements Aut
                 throw new IllegalArgumentException("CoordinatorMetrics must be set.");
             if (serializer == null)
                 throw new IllegalArgumentException("Serializer must be set.");
-            if (compressionType == null)
-                compressionType = CompressionType.NONE;
+            if (compression == null)
+                compression = Compression.NONE;
 
             return new CoordinatorRuntime<>(
                 logPrefix,
@@ -220,7 +220,7 @@ public class CoordinatorRuntime, U> implements Aut
                 runtimeMetrics,
                 coordinatorMetrics,
                 serializer,
-                compressionType
+                compression
             );
         }
     }
@@ -793,7 +793,7 @@ public class CoordinatorRuntime, U> implements Aut
                             MemoryRecordsBuilder builder = MemoryRecords.builder(
                                 buffer,
                                 magic,
-                                compressionType,
+                                compression,
                                 TimestampType.CREATE_TIME,
                                 0L,
                                 currentTimeMs,
@@ -1426,7 +1426,7 @@ public class CoordinatorRuntime, U> implements Aut
     /**
      * The compression codec used when writing records.
      */
-    private final CompressionType compressionType;
+    private final Compression compression;
 
     /**
      * Atomic boolean indicating whether the runtime is running.
@@ -1453,7 +1453,7 @@ public class CoordinatorRuntime, U> implements Aut
      * @param runtimeMetrics                    The runtime metrics.
      * @param coordinatorMetrics                The coordinator metrics.
      * @param serializer                        The serializer.
-     * @param compressionType                   The compression type.
+     * @param compression                       The compression codec.
      */
     private CoordinatorRuntime(
         String logPrefix,
@@ -1468,7 +1468,7 @@ public class CoordinatorRuntime, U> implements Aut
         CoordinatorRuntimeMetrics runtimeMetrics,
         CoordinatorMetrics coordinatorMetrics,
         Serializer serializer,
-        CompressionType compressionType
+        Compression compression
     ) {
         this.logPrefix = logPrefix;
         this.logContext = logContext;
@@ -1484,7 +1484,7 @@ public class CoordinatorRuntime, U> implements Aut
         this.runtimeMetrics = runtimeMetrics;
         this.coordinatorMetrics = coordinatorMetrics;
         this.serializer = serializer;
-        this.compressionType = compressionType;
+        this.compression = compression;
     }
 
     /**
diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntimeTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntimeTest.java
index 1be63c58ccf..d874ceae95f 100644
--- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntimeTest.java
+++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/runtime/CoordinatorRuntimeTest.java
@@ -18,6 +18,7 @@ package org.apache.kafka.coordinator.group.runtime;
 
 import org.apache.kafka.common.KafkaException;
 import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.compress.Compression;
 import org.apache.kafka.common.errors.NotCoordinatorException;
 import org.apache.kafka.common.errors.NotEnoughReplicasException;
 import org.apache.kafka.common.protocol.ApiKeys;
@@ -423,7 +424,7 @@ public class CoordinatorRuntimeTest {
         MemoryRecordsBuilder builder = MemoryRecords.builder(
             buffer,
             RecordVersion.current().value,
-            CompressionType.NONE,
+            Compression.NONE,
             TimestampType.CREATE_TIME,
             0L,
             timestamp,
@@ -463,7 +464,7 @@ public class CoordinatorRuntimeTest {
         MemoryRecordsBuilder builder = MemoryRecords.builder(
             buffer,
             RecordVersion.current().value,
-            CompressionType.NONE,
+            Compression.NONE,
             TimestampType.CREATE_TIME,
             0L,
             timestamp,
diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/FetchResponseBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/FetchResponseBenchmark.java
index d8512bdd0df..0320310606c 100644
--- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/FetchResponseBenchmark.java
+++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/common/FetchResponseBenchmark.java
@@ -20,11 +20,11 @@ package org.apache.kafka.jmh.common;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.TopicIdPartition;
 import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.compress.Compression;
 import org.apache.kafka.common.message.FetchResponseData;
 import org.apache.kafka.common.network.Send;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.Errors;
-import org.apache.kafka.common.record.CompressionType;
 import org.apache.kafka.common.record.MemoryRecords;
 import org.apache.kafka.common.record.SimpleRecord;
 import org.apache.kafka.common.requests.ByteBufferChannel;
@@ -78,7 +78,7 @@ public class FetchResponseBenchmark {
 
     @Setup(Level.Trial)
     public void setup() {
-        MemoryRecords records = MemoryRecords.withRecords(CompressionType.NONE,
+        MemoryRecords records = MemoryRecords.withRecords(Compression.NONE,
                 new SimpleRecord(1000, "key1".getBytes(StandardCharsets.UTF_8), "value1".getBytes(StandardCharsets.UTF_8)),
                 new SimpleRecord(1001, "key2".getBytes(StandardCharsets.UTF_8), "value2".getBytes(StandardCharsets.UTF_8)),
                 new SimpleRecord(1002, "key3".getBytes(StandardCharsets.UTF_8), "value3".getBytes(StandardCharsets.UTF_8)));
diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/partition/PartitionMakeFollowerBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/partition/PartitionMakeFollowerBenchmark.java
index 0f041685673..8d016b504e8 100644
--- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/partition/PartitionMakeFollowerBenchmark.java
+++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/partition/PartitionMakeFollowerBenchmark.java
@@ -29,8 +29,8 @@ import kafka.server.checkpoints.OffsetCheckpoints;
 import kafka.server.metadata.MockConfigRepository;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.compress.Compression;
 import org.apache.kafka.common.message.LeaderAndIsrRequestData;
-import org.apache.kafka.common.record.CompressionType;
 import org.apache.kafka.common.record.MemoryRecords;
 import org.apache.kafka.common.record.SimpleRecord;
 import org.apache.kafka.common.utils.Time;
@@ -133,7 +133,7 @@ public class PartitionMakeFollowerBenchmark {
             };
             int initialOffSet = 0;
             while (true) {
-                MemoryRecords memoryRecords =  MemoryRecords.withRecords(initialOffSet, CompressionType.NONE, 0, simpleRecords);
+                MemoryRecords memoryRecords =  MemoryRecords.withRecords(initialOffSet, Compression.NONE, 0, simpleRecords);
                 partition.appendRecordsToFollowerOrFutureReplica(memoryRecords, false);
                 initialOffSet = initialOffSet + 2;
             }
diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerRequestBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerRequestBenchmark.java
index 22d49551dd5..4ea35a563f3 100644
--- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerRequestBenchmark.java
+++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/producer/ProducerRequestBenchmark.java
@@ -17,9 +17,9 @@
 
 package org.apache.kafka.jmh.producer;
 
+import org.apache.kafka.common.compress.Compression;
 import org.apache.kafka.common.message.ProduceRequestData;
 import org.apache.kafka.common.protocol.Errors;
-import org.apache.kafka.common.record.CompressionType;
 import org.apache.kafka.common.record.MemoryRecords;
 import org.apache.kafka.common.record.RecordBatch;
 import org.apache.kafka.common.record.SimpleRecord;
@@ -55,7 +55,7 @@ public class ProducerRequestBenchmark {
             .setName("tp")
             .setPartitionData(IntStream.range(0, NUMBER_OF_PARTITIONS).mapToObj(partitionIndex -> new ProduceRequestData.PartitionProduceData()
                 .setIndex(partitionIndex)
-                .setRecords(MemoryRecords.withRecords(CompressionType.NONE, IntStream.range(0, NUMBER_OF_RECORDS)
+                .setRecords(MemoryRecords.withRecords(Compression.NONE, IntStream.range(0, NUMBER_OF_RECORDS)
                     .mapToObj(recordIndex -> new SimpleRecord(100, "hello0".getBytes(StandardCharsets.UTF_8)))
                     .collect(Collectors.toList())
                     .toArray(new SimpleRecord[0]))))
diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/BaseRecordBatchBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/BaseRecordBatchBenchmark.java
index ade2a8fe007..0a9c5ff5d44 100644
--- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/BaseRecordBatchBenchmark.java
+++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/BaseRecordBatchBenchmark.java
@@ -19,9 +19,9 @@ package org.apache.kafka.jmh.record;
 import kafka.log.UnifiedLog;
 import kafka.server.BrokerTopicStats;
 import kafka.server.RequestLocal;
+import org.apache.kafka.common.compress.Compression;
 import org.apache.kafka.common.header.Header;
 import org.apache.kafka.common.record.AbstractRecords;
-import org.apache.kafka.common.record.CompressionType;
 import org.apache.kafka.common.record.MemoryRecords;
 import org.apache.kafka.common.record.MemoryRecordsBuilder;
 import org.apache.kafka.common.record.Record;
@@ -129,19 +129,19 @@ public abstract class BaseRecordBatchBenchmark {
         }).toArray(Header[]::new);
     }
 
-    abstract CompressionType compressionType();
+    abstract Compression compression();
 
     private ByteBuffer createBatch(int batchSize) {
         // Magic v1 does not support record headers
         Header[] headers = messageVersion < RecordBatch.MAGIC_VALUE_V2 ? Record.EMPTY_HEADERS : createHeaders();
         byte[] value = new byte[messageSize];
         final ByteBuffer buf = ByteBuffer.allocate(
-            AbstractRecords.estimateSizeInBytesUpperBound(messageVersion, compressionType(), new byte[0], value,
+            AbstractRecords.estimateSizeInBytesUpperBound(messageVersion, compression().type(), new byte[0], value,
                     headers) * batchSize
         );
 
         final MemoryRecordsBuilder builder =
-            MemoryRecords.builder(buf, messageVersion, compressionType(), TimestampType.CREATE_TIME, startingOffset);
+            MemoryRecords.builder(buf, messageVersion, compression(), TimestampType.CREATE_TIME, startingOffset);
 
         for (int i = 0; i < batchSize; ++i) {
             switch (bytes) {
diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/CompressedRecordBatchValidationBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/CompressedRecordBatchValidationBenchmark.java
index 7cd8719bf59..b43d7a9271e 100644
--- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/CompressedRecordBatchValidationBenchmark.java
+++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/CompressedRecordBatchValidationBenchmark.java
@@ -17,6 +17,7 @@
 package org.apache.kafka.jmh.record;
 
 import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.compress.Compression;
 import org.apache.kafka.common.record.CompressionType;
 import org.apache.kafka.common.record.MemoryRecords;
 import org.apache.kafka.common.record.TimestampType;
@@ -44,15 +45,15 @@ public class CompressedRecordBatchValidationBenchmark extends BaseRecordBatchBen
     private CompressionType compressionType = CompressionType.LZ4;
 
     @Override
-    CompressionType compressionType() {
-        return compressionType;
+    Compression compression() {
+        return Compression.of(compressionType).build();
     }
 
     @Benchmark
     public void measureValidateMessagesAndAssignOffsetsCompressed(Blackhole bh) {
         MemoryRecords records = MemoryRecords.readableRecords(singleBatchBuffer.duplicate());
         new LogValidator(records, new TopicPartition("a", 0),
-            Time.SYSTEM, compressionType, compressionType, false,  messageVersion,
+            Time.SYSTEM, compressionType, compression(), false,  messageVersion,
             TimestampType.CREATE_TIME, Long.MAX_VALUE, Long.MAX_VALUE, 0, AppendOrigin.CLIENT,
             MetadataVersion.latestTesting()
         ).validateMessagesAndAssignOffsetsCompressed(PrimitiveRef.ofLong(startingOffset),
diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/RecordBatchIterationBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/RecordBatchIterationBenchmark.java
index c331cd58404..d8a73d53d30 100644
--- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/RecordBatchIterationBenchmark.java
+++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/RecordBatchIterationBenchmark.java
@@ -16,6 +16,7 @@
  */
 package org.apache.kafka.jmh.record;
 
+import org.apache.kafka.common.compress.Compression;
 import org.apache.kafka.common.record.CompressionType;
 import org.apache.kafka.common.record.MemoryRecords;
 import org.apache.kafka.common.record.MutableRecordBatch;
@@ -42,8 +43,8 @@ public class RecordBatchIterationBenchmark extends BaseRecordBatchBenchmark {
     private CompressionType compressionType = CompressionType.NONE;
 
     @Override
-    CompressionType compressionType() {
-        return compressionType;
+    Compression compression() {
+        return Compression.of(compressionType).build();
     }
 
     @Benchmark
diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/UncompressedRecordBatchValidationBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/UncompressedRecordBatchValidationBenchmark.java
index 18e73eae2e1..66f74c70467 100644
--- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/UncompressedRecordBatchValidationBenchmark.java
+++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/record/UncompressedRecordBatchValidationBenchmark.java
@@ -17,6 +17,7 @@
 package org.apache.kafka.jmh.record;
 
 import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.compress.Compression;
 import org.apache.kafka.common.record.CompressionType;
 import org.apache.kafka.common.record.MemoryRecords;
 import org.apache.kafka.common.record.TimestampType;
@@ -40,15 +41,15 @@ import org.openjdk.jmh.infra.Blackhole;
 public class UncompressedRecordBatchValidationBenchmark extends BaseRecordBatchBenchmark {
 
     @Override
-    CompressionType compressionType() {
-        return CompressionType.NONE;
+    Compression compression() {
+        return Compression.NONE;
     }
 
     @Benchmark
     public void measureAssignOffsetsNonCompressed(Blackhole bh) {
         MemoryRecords records = MemoryRecords.readableRecords(singleBatchBuffer.duplicate());
         new LogValidator(records, new TopicPartition("a", 0),
-            Time.SYSTEM, CompressionType.NONE, CompressionType.NONE, false,
+            Time.SYSTEM, CompressionType.NONE, Compression.NONE, false,
             messageVersion, TimestampType.CREATE_TIME, Long.MAX_VALUE, Long.MAX_VALUE, 0, AppendOrigin.CLIENT,
             MetadataVersion.latestTesting()
         ).assignOffsetsNonCompressed(PrimitiveRef.ofLong(startingOffset), validatorMetricsRecorder);
diff --git a/metadata/src/main/java/org/apache/kafka/metadata/util/BatchFileWriter.java b/metadata/src/main/java/org/apache/kafka/metadata/util/BatchFileWriter.java
index 0ccf39a85ac..9466a4f3f77 100644
--- a/metadata/src/main/java/org/apache/kafka/metadata/util/BatchFileWriter.java
+++ b/metadata/src/main/java/org/apache/kafka/metadata/util/BatchFileWriter.java
@@ -17,9 +17,9 @@
 
 package org.apache.kafka.metadata.util;
 
+import org.apache.kafka.common.compress.Compression;
 import org.apache.kafka.common.message.SnapshotFooterRecord;
 import org.apache.kafka.common.message.SnapshotHeaderRecord;
-import org.apache.kafka.common.record.CompressionType;
 import org.apache.kafka.common.record.ControlRecordUtils;
 import org.apache.kafka.common.utils.Time;
 import org.apache.kafka.common.utils.Utils;
@@ -97,7 +97,7 @@ public class BatchFileWriter implements AutoCloseable {
             MAX_BATCH_SIZE_BYTES,
             new BatchMemoryPool(5, MAX_BATCH_SIZE_BYTES),
             time,
-            CompressionType.NONE,
+            Compression.NONE,
             new MetadataRecordSerde()
         );
 
diff --git a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java
index 288933b9c6c..ed09a1449e7 100644
--- a/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java
+++ b/raft/src/main/java/org/apache/kafka/raft/KafkaRaftClient.java
@@ -19,6 +19,7 @@ package org.apache.kafka.raft;
 import org.apache.kafka.common.KafkaException;
 import org.apache.kafka.common.TopicPartition;
 import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.compress.Compression;
 import org.apache.kafka.common.errors.ClusterAuthorizationException;
 import org.apache.kafka.common.errors.NotLeaderOrFollowerException;
 import org.apache.kafka.common.memory.MemoryPool;
@@ -38,7 +39,6 @@ import org.apache.kafka.common.metrics.Metrics;
 import org.apache.kafka.common.protocol.ApiKeys;
 import org.apache.kafka.common.protocol.ApiMessage;
 import org.apache.kafka.common.protocol.Errors;
-import org.apache.kafka.common.record.CompressionType;
 import org.apache.kafka.common.record.MemoryRecords;
 import org.apache.kafka.common.record.Records;
 import org.apache.kafka.common.record.UnalignedMemoryRecords;
@@ -485,7 +485,7 @@ final public class KafkaRaftClient