diff --git a/CHANGELOG.md b/CHANGELOG.md
index 09f9ad84e..52fdb116a 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -7,6 +7,7 @@ accidentally triggering the load of a previous DB version.**
 ## Unreleased
 
 **Bugfixes**
+* #4926 Fix corruption when inserting into compressed chunks
 * #5101 Fix enabling compression on caggs with renamed columns
 
 ## 2.9.0 (2022-12-15)
diff --git a/sql/policy_internal.sql b/sql/policy_internal.sql
index c0983203c..9ec7baf33 100644
--- a/sql/policy_internal.sql
+++ b/sql/policy_internal.sql
@@ -49,6 +49,11 @@ DECLARE
   numchunks   INTEGER := 1;
   _message     text;
   _detail      text;
+  -- chunk status bits:
+  bit_compressed int := 1;
+  bit_compressed_unordered int := 2;
+  bit_frozen int := 4;
+  bit_compressed_partial int := 8;
 BEGIN
 
   -- procedures with SET clause cannot execute transaction
@@ -75,7 +80,15 @@ BEGIN
       INNER JOIN _timescaledb_catalog.chunk ch ON ch.table_name = pgc.relname AND ch.schema_name = pgns.nspname AND ch.hypertable_id = htid
     WHERE
       ch.dropped IS FALSE
-      AND (ch.status = 0 OR ch.status = 3)
+      AND (
+        ch.status = 0 OR
+        (
+          ch.status & bit_compressed > 0 AND (
+            ch.status & bit_compressed_unordered > 0 OR
+            ch.status & bit_compressed_partial > 0
+          )
+        )
+      )
   LOOP
     IF chunk_rec.status = 0 THEN
       BEGIN
@@ -88,7 +101,13 @@ BEGIN
             USING DETAIL = format('Message: (%s), Detail: (%s).', _message, _detail),
                   ERRCODE = sqlstate;
       END;
-    ELSIF chunk_rec.status = 3 AND recompress_enabled IS TRUE THEN
+    ELSIF
+      (
+        chunk_rec.status & bit_compressed > 0 AND (
+          chunk_rec.status & bit_compressed_unordered > 0 OR
+          chunk_rec.status & bit_compressed_partial > 0
+        )
+      ) AND recompress_enabled IS TRUE THEN
       BEGIN
         PERFORM @extschema@.decompress_chunk(chunk_rec.oid, if_compressed => true);
       EXCEPTION WHEN OTHERS THEN
diff --git a/src/chunk.c b/src/chunk.c
index 715278c22..c0b060751 100644
--- a/src/chunk.c
+++ b/src/chunk.c
@@ -4351,9 +4351,11 @@ ts_chunk_get_compression_status(int32 chunk_id)
 				ts_flags_are_set_32(DatumGetInt32(status), CHUNK_STATUS_COMPRESSED);
 			bool status_is_unordered =
 				ts_flags_are_set_32(DatumGetInt32(status), CHUNK_STATUS_COMPRESSED_UNORDERED);
+			bool status_is_partial =
+				ts_flags_are_set_32(DatumGetInt32(status), CHUNK_STATUS_COMPRESSED_PARTIAL);
 			if (status_is_compressed)
 			{
-				if (status_is_unordered)
+				if (status_is_unordered || status_is_partial)
 					st = CHUNK_COMPRESS_UNORDERED;
 				else
 					st = CHUNK_COMPRESS_ORDERED;
diff --git a/src/copy.c b/src/copy.c
index 6112f68b2..103bbcafd 100644
--- a/src/copy.c
+++ b/src/copy.c
@@ -712,13 +712,6 @@ copyfrom(CopyChunkState *ccstate, List *range_table, Hypertable *ht, MemoryConte
 {
 	ResultRelInfo *resultRelInfo;
 	ResultRelInfo *saved_resultRelInfo = NULL;
-	/* if copies are directed to a chunk that is compressed, we redirect
-	 * them to the internal compressed chunk. But we still
-	 * need to check triggers, constrainst etc. against the original
-	 * chunk (not the internal compressed chunk).
-	 * check_resultRelInfo saves that information
-	 */
-	ResultRelInfo *check_resultRelInfo = NULL;
 	EState *estate = ccstate->estate; /* for ExecConstraints() */
 	ExprContext *econtext;
 	TupleTableSlot *singleslot;
@@ -965,10 +958,6 @@ copyfrom(CopyChunkState *ccstate, List *range_table, Hypertable *ht, MemoryConte
 
 		currentTupleInsertMethod = insertMethod;
 
-		/* Insert tuples into compressed chunks tuple by tuple */
-		if (cis->compress_info)
-			currentTupleInsertMethod = CIM_SINGLE;
-
 		/* Determine which triggers exist on this chunk */
 		has_before_insert_row_trig =
 			(cis->result_relation_info->ri_TrigDesc &&
@@ -1047,21 +1036,14 @@ copyfrom(CopyChunkState *ccstate, List *range_table, Hypertable *ht, MemoryConte
 		estate->es_result_relation_info = resultRelInfo;
 #endif
 
-		if (cis->compress_info != NULL)
-			check_resultRelInfo = cis->compress_info->orig_result_relation_info;
-		else
-			check_resultRelInfo = resultRelInfo;
-
 		/* Set the right relation for triggers */
-		ts_tuptableslot_set_table_oid(myslot,
-									  RelationGetRelid(check_resultRelInfo->ri_RelationDesc));
+		ts_tuptableslot_set_table_oid(myslot, RelationGetRelid(resultRelInfo->ri_RelationDesc));
 
 		skip_tuple = false;
 
 		/* BEFORE ROW INSERT Triggers */
-		if (check_resultRelInfo->ri_TrigDesc &&
-			check_resultRelInfo->ri_TrigDesc->trig_insert_before_row)
-			skip_tuple = !ExecBRInsertTriggers(estate, check_resultRelInfo, myslot);
+		if (resultRelInfo->ri_TrigDesc && resultRelInfo->ri_TrigDesc->trig_insert_before_row)
+			skip_tuple = !ExecBRInsertTriggers(estate, resultRelInfo, myslot);
 
 		if (!skip_tuple)
 		{
@@ -1072,110 +1054,72 @@ copyfrom(CopyChunkState *ccstate, List *range_table, Hypertable *ht, MemoryConte
 			List *recheckIndexes = NIL;
 
 			/* Compute stored generated columns */
-			if (check_resultRelInfo->ri_RelationDesc->rd_att->constr &&
-				check_resultRelInfo->ri_RelationDesc->rd_att->constr->has_generated_stored)
-				ExecComputeStoredGeneratedCompat(check_resultRelInfo, estate, myslot, CMD_INSERT);
+			if (resultRelInfo->ri_RelationDesc->rd_att->constr &&
+				resultRelInfo->ri_RelationDesc->rd_att->constr->has_generated_stored)
+				ExecComputeStoredGeneratedCompat(resultRelInfo, estate, myslot, CMD_INSERT);
 
 			/*
 			 * If the target is a plain table, check the constraints of
 			 * the tuple.
 			 */
-			if (check_resultRelInfo->ri_FdwRoutine == NULL &&
-				check_resultRelInfo->ri_RelationDesc->rd_att->constr)
+			if (resultRelInfo->ri_FdwRoutine == NULL &&
+				resultRelInfo->ri_RelationDesc->rd_att->constr)
 			{
-				Assert(check_resultRelInfo->ri_RangeTableIndex > 0 && estate->es_range_table);
-				ExecConstraints(check_resultRelInfo, myslot, estate);
+				Assert(resultRelInfo->ri_RangeTableIndex > 0 && estate->es_range_table);
+				ExecConstraints(resultRelInfo, myslot, estate);
 			}
 
-			if (cis->compress_info)
+			if (currentTupleInsertMethod == CIM_SINGLE)
 			{
-				Assert(currentTupleInsertMethod == CIM_SINGLE);
-
-				TupleTableSlot *compress_slot =
-					ts_cm_functions->compress_row_exec(cis->compress_info->compress_state, myslot);
-				/* After Row triggers do not work with compressed chunks. So
-				 * explicitly call cagg trigger here
-				 */
-				if (cis->compress_info->has_cagg_trigger)
-				{
-					HeapTupleTableSlot *hslot = (HeapTupleTableSlot *) myslot;
-					if (!hslot->tuple)
-						hslot->tuple = heap_form_tuple(myslot->tts_tupleDescriptor,
-													   myslot->tts_values,
-													   myslot->tts_isnull);
-					ts_compress_chunk_invoke_cagg_trigger(cis->compress_info,
-														  cis->rel,
-														  hslot->tuple);
-				}
-
+				/* OK, store the tuple and create index entries for it */
 				table_tuple_insert(resultRelInfo->ri_RelationDesc,
-								   compress_slot,
+								   myslot,
 								   mycid,
 								   ti_options,
 								   bistate);
+
 				if (resultRelInfo->ri_NumIndices > 0)
 					recheckIndexes = ExecInsertIndexTuplesCompat(resultRelInfo,
-																 compress_slot,
+																 myslot,
 																 estate,
 																 false,
 																 false,
 																 NULL,
 																 NIL);
+				/* AFTER ROW INSERT Triggers */
+				ExecARInsertTriggers(estate,
+									 resultRelInfo,
+									 myslot,
+									 recheckIndexes,
+									 NULL /* transition capture */);
 			}
 			else
 			{
-				if (currentTupleInsertMethod == CIM_SINGLE)
+				/*
+				 * The slot previously might point into the per-tuple
+				 * context. For batching it needs to be longer lived.
+				 */
+				ExecMaterializeSlot(myslot);
+
+				/* Add this tuple to the tuple buffer */
+				TSCopyMultiInsertInfoStore(&multiInsertInfo,
+										   resultRelInfo,
+										   buffer,
+										   myslot,
+										   ccstate->cstate);
+
+				/*
+				 * If enough inserts have queued up, then flush all
+				 * buffers out to their tables.
+				 */
+				if (TSCopyMultiInsertInfoIsFull(&multiInsertInfo))
 				{
-					/* OK, store the tuple and create index entries for it */
-					table_tuple_insert(resultRelInfo->ri_RelationDesc,
-									   myslot,
-									   mycid,
-									   ti_options,
-									   bistate);
+					ereport(DEBUG2,
+							(errmsg("flush called with %d bytes and %d buffered tuples",
+									multiInsertInfo.bufferedBytes,
+									multiInsertInfo.bufferedTuples)));
 
-					if (resultRelInfo->ri_NumIndices > 0)
-						recheckIndexes = ExecInsertIndexTuplesCompat(resultRelInfo,
-																	 myslot,
-																	 estate,
-																	 false,
-																	 false,
-																	 NULL,
-																	 NIL);
-					/* AFTER ROW INSERT Triggers */
-					ExecARInsertTriggers(estate,
-										 check_resultRelInfo,
-										 myslot,
-										 recheckIndexes,
-										 NULL /* transition capture */);
-				}
-				else
-				{
-					/*
-					 * The slot previously might point into the per-tuple
-					 * context. For batching it needs to be longer lived.
-					 */
-					ExecMaterializeSlot(myslot);
-
-					/* Add this tuple to the tuple buffer */
-					TSCopyMultiInsertInfoStore(&multiInsertInfo,
-											   resultRelInfo,
-											   buffer,
-											   myslot,
-											   ccstate->cstate);
-
-					/*
-					 * If enough inserts have queued up, then flush all
-					 * buffers out to their tables.
-					 */
-					if (TSCopyMultiInsertInfoIsFull(&multiInsertInfo))
-					{
-						ereport(DEBUG2,
-								(errmsg("Flush called with %d bytes and %d buffered tuples",
-										multiInsertInfo.bufferedBytes,
-										multiInsertInfo.bufferedTuples)));
-
-						TSCopyMultiInsertInfoFlush(&multiInsertInfo, cis);
-					}
+					TSCopyMultiInsertInfoFlush(&multiInsertInfo, cis);
 				}
 			}
 
diff --git a/src/nodes/chunk_dispatch_state.c b/src/nodes/chunk_dispatch_state.c
index 1bc1e5973..79b932536 100644
--- a/src/nodes/chunk_dispatch_state.c
+++ b/src/nodes/chunk_dispatch_state.c
@@ -115,10 +115,7 @@ chunk_dispatch_exec(CustomScanState *node)
 	 * just when the chunk changes.
 	 */
 #if PG14_LT
-	if (cis->compress_info != NULL)
-		estate->es_result_relation_info = cis->compress_info->orig_result_relation_info;
-	else
-		estate->es_result_relation_info = cis->result_relation_info;
+	estate->es_result_relation_info = cis->result_relation_info;
 #endif
 
 	MemoryContextSwitchTo(old);
@@ -127,57 +124,6 @@ chunk_dispatch_exec(CustomScanState *node)
 	if (cis->hyper_to_chunk_map != NULL)
 		slot = execute_attr_map_slot(cis->hyper_to_chunk_map->attrMap, slot, cis->slot);
 
-	if (cis->compress_info != NULL)
-	{
-		/*
-		 * When the chunk is compressed, we redirect the insert to the internal compressed
-		 * chunk. However, any BEFORE ROW triggers defined on the chunk have to be executed
-		 * before we redirect the insert.
-		 */
-		if (cis->compress_info->orig_result_relation_info->ri_TrigDesc &&
-			cis->compress_info->orig_result_relation_info->ri_TrigDesc->trig_insert_before_row)
-		{
-			bool skip_tuple;
-			skip_tuple =
-				!ExecBRInsertTriggers(estate, cis->compress_info->orig_result_relation_info, slot);
-
-			if (skip_tuple)
-				return NULL;
-		}
-
-		if (cis->rel->rd_att->constr && cis->rel->rd_att->constr->has_generated_stored)
-			ExecComputeStoredGeneratedCompat(cis->compress_info->orig_result_relation_info,
-											 estate,
-											 slot,
-											 CMD_INSERT);
-
-		if (cis->rel->rd_att->constr)
-			ExecConstraints(cis->compress_info->orig_result_relation_info, slot, estate);
-
-#if PG14_LT
-		estate->es_result_relation_info = cis->result_relation_info;
-#endif
-		Assert(ts_cm_functions->compress_row_exec != NULL);
-		TupleTableSlot *orig_slot = slot;
-		old = MemoryContextSwitchTo(GetPerTupleMemoryContext(estate));
-		slot = ts_cm_functions->compress_row_exec(cis->compress_info->compress_state, slot);
-		MemoryContextSwitchTo(old);
-		/* If we have cagg defined on the hypertable, we have to execute
-		 * the function that records invalidations directly as AFTER ROW
-		 * triggers do not work with compressed chunks.
-		 */
-		if (cis->compress_info->has_cagg_trigger)
-		{
-			Assert(ts_cm_functions->continuous_agg_call_invalidation_trigger);
-			HeapTupleTableSlot *hslot = (HeapTupleTableSlot *) orig_slot;
-			if (!hslot->tuple)
-				hslot->tuple = heap_form_tuple(orig_slot->tts_tupleDescriptor,
-											   orig_slot->tts_values,
-											   orig_slot->tts_isnull);
-
-			ts_compress_chunk_invoke_cagg_trigger(cis->compress_info, cis->rel, hslot->tuple);
-		}
-	}
 	return slot;
 }
 
diff --git a/src/nodes/chunk_insert_state.c b/src/nodes/chunk_insert_state.c
index 3597d8d85..2be07e85c 100644
--- a/src/nodes/chunk_insert_state.c
+++ b/src/nodes/chunk_insert_state.c
@@ -111,32 +111,6 @@ create_chunk_result_relation_info(ChunkDispatch *dispatch, Relation rel)
 	return rri;
 }
 
-static inline ResultRelInfo *
-create_compress_chunk_result_relation_info(ChunkDispatch *dispatch, Relation compress_rel)
-{
-	ResultRelInfo *rri = makeNode(ResultRelInfo);
-	ResultRelInfo *rri_orig = dispatch->hypertable_result_rel_info;
-	Index hyper_rti = rri_orig->ri_RangeTableIndex;
-
-	InitResultRelInfo(rri, compress_rel, hyper_rti, NULL, dispatch->estate->es_instrument);
-
-	/* RLS policies are not supported if compression is enabled */
-	Assert(rri_orig->ri_WithCheckOptions == NULL && rri_orig->ri_WithCheckOptionExprs == NULL);
-	Assert(rri_orig->ri_projectReturning == NULL);
-#if PG14_LT
-	rri->ri_junkFilter = rri_orig->ri_junkFilter;
-#endif
-
-	/* compressed rel chunk is on data node. Does not need any FDW access on AN */
-	rri->ri_FdwState = NULL;
-	rri->ri_usesFdwDirectModify = false;
-	rri->ri_FdwRoutine = NULL;
-	/* constraints are executed on the orig base chunk. So we do
-	 * not call create_chunk_rri_constraint_expr here
-	 */
-	return rri;
-}
-
 static ProjectionInfo *
 get_adjusted_projection_info_returning(ProjectionInfo *orig, List *returning_clauses,
 									   TupleConversionMap *map, Index varno, Oid rowtype,
@@ -542,32 +516,6 @@ adjust_projections(ChunkInsertState *cis, ChunkDispatch *dispatch, Oid rowtype)
 	}
 }
 
-/* Assumption: we have acquired a lock on the chunk table. This is
- *      important because lock acquisition order is always orignal chunk,
- *      followed by compressed chunk to prevent deadlocks.
- * We now try to acquire a lock on the compressed chunk, if one exists.
- * Note that the insert could have been blocked by a recompress_chunk operation.
- * So the compressed chunk could have moved under us. We need to refetch the chunk
- * to get the correct compressed chunk id (github issue 3400)
- */
-static Relation
-lock_associated_compressed_chunk(int32 chunk_id, bool *has_compressed_chunk)
-{
-	int32 compressed_chunk_id = ts_chunk_get_compressed_chunk_id(chunk_id);
-	if (compressed_chunk_id)
-	{
-		Oid compress_chunk_relid =
-			ts_chunk_get_relid(compressed_chunk_id, /* missing_ok = */ false);
-		Assert(OidIsValid(compress_chunk_relid));
-
-		*has_compressed_chunk = true;
-		return table_open(compress_chunk_relid, RowExclusiveLock);
-	}
-
-	*has_compressed_chunk = false;
-	return NULL;
-}
-
 /*
  * Create new insert chunk state.
  *
@@ -577,15 +525,13 @@ lock_associated_compressed_chunk(int32 chunk_id, bool *has_compressed_chunk)
 extern ChunkInsertState *
 ts_chunk_insert_state_create(const Chunk *chunk, ChunkDispatch *dispatch)
 {
-	int cagg_trig_nargs = 0;
-	int32 cagg_trig_args[2] = { 0, 0 };
 	ChunkInsertState *state;
-	Relation rel, parent_rel, compress_rel = NULL;
+	Relation rel, parent_rel;
 	MemoryContext cis_context = AllocSetContextCreate(dispatch->estate->es_query_cxt,
 													  "chunk insert state memory context",
 													  ALLOCSET_DEFAULT_SIZES);
 	OnConflictAction onconflict_action = ts_chunk_dispatch_get_on_conflict_action(dispatch);
-	ResultRelInfo *resrelinfo, *relinfo;
+	ResultRelInfo *relinfo;
 	bool has_compressed_chunk = (chunk->fd.compressed_chunk_id != 0);
 
 	/* permissions NOT checked here; were checked at hypertable level */
@@ -593,9 +539,7 @@ ts_chunk_insert_state_create(const Chunk *chunk, ChunkDispatch *dispatch)
 		ereport(ERROR,
 				(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
 				 errmsg("hypertables do not support row-level security")));
-
-	if (chunk->relkind != RELKIND_RELATION && chunk->relkind != RELKIND_FOREIGN_TABLE)
-		elog(ERROR, "insert is not on a table");
+	Assert(chunk->relkind == RELKIND_RELATION || chunk->relkind == RELKIND_FOREIGN_TABLE);
 
 	ts_chunk_validate_chunk_status_for_operation(chunk->table_id,
 												 chunk->fd.status,
@@ -617,28 +561,23 @@ ts_chunk_insert_state_create(const Chunk *chunk, ChunkDispatch *dispatch)
 				 errmsg("insert into a compressed chunk that has primary or unique constraint is "
 						"not supported")));
 	}
-	compress_rel = lock_associated_compressed_chunk(chunk->fd.id, &has_compressed_chunk);
 
 	MemoryContext old_mcxt = MemoryContextSwitchTo(cis_context);
 	relinfo = create_chunk_result_relation_info(dispatch, rel);
-	if (!has_compressed_chunk)
-		resrelinfo = relinfo;
-	else
-	{
-		/* insert the tuple into the compressed chunk instead */
-		resrelinfo = create_compress_chunk_result_relation_info(dispatch, compress_rel);
-	}
-	CheckValidResultRel(resrelinfo, ts_chunk_dispatch_get_cmd_type(dispatch));
+	CheckValidResultRel(relinfo, ts_chunk_dispatch_get_cmd_type(dispatch));
 
 	state = palloc0(sizeof(ChunkInsertState));
 	state->mctx = cis_context;
 	state->rel = rel;
-	state->result_relation_info = resrelinfo;
+	state->result_relation_info = relinfo;
 	state->estate = dispatch->estate;
 
-	if (resrelinfo->ri_RelationDesc->rd_rel->relhasindex &&
-		resrelinfo->ri_IndexRelationDescs == NULL)
-		ExecOpenIndices(resrelinfo, onconflict_action != ONCONFLICT_NONE);
+	state->chunk_compressed = ts_chunk_is_compressed(chunk);
+	if (state->chunk_compressed)
+		state->chunk_partial = ts_chunk_is_partial(chunk);
+
+	if (relinfo->ri_RelationDesc->rd_rel->relhasindex && relinfo->ri_IndexRelationDescs == NULL)
+		ExecOpenIndices(relinfo, onconflict_action != ONCONFLICT_NONE);
 
 	if (relinfo->ri_TrigDesc != NULL)
 	{
@@ -658,42 +597,6 @@ ts_chunk_insert_state_create(const Chunk *chunk, ChunkDispatch *dispatch)
 		 */
 		if (tg->trig_insert_after_statement || tg->trig_insert_before_statement)
 			elog(ERROR, "statement trigger on chunk table not supported");
-
-		/* AFTER ROW triggers do not work since we redirect the insert
-		 * to the compressed chunk. We still want cagg triggers to fire.
-		 * We'll call them directly. But raise an error if there are
-		 * other triggers
-		 */
-		if (has_compressed_chunk && tg->trig_insert_after_row)
-		{
-			StringInfo trigger_list = makeStringInfo();
-			Assert(tg->numtriggers > 0);
-			for (int i = 0; i < tg->numtriggers; i++)
-			{
-				if (strncmp(tg->triggers[i].tgname,
-							CAGGINVAL_TRIGGER_NAME,
-							strlen(CAGGINVAL_TRIGGER_NAME)) == 0)
-				{
-					/* collect arg information here */
-					cagg_trig_nargs = tg->triggers[i].tgnargs;
-					cagg_trig_args[0] = atol(tg->triggers[i].tgargs[0]);
-					if (cagg_trig_nargs > 1)
-						cagg_trig_args[1] = atol(tg->triggers[i].tgargs[1]);
-					continue;
-				}
-				if (i > 0)
-					appendStringInfoString(trigger_list, ", ");
-				appendStringInfoString(trigger_list, tg->triggers[i].tgname);
-			}
-			if (trigger_list->len != 0)
-			{
-				ereport(ERROR,
-						(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
-						 errmsg("after insert row trigger on compressed chunk not supported"),
-						 errdetail("Triggers: %s", trigger_list->data),
-						 errhint("Decompress the chunk first before inserting into it.")));
-			}
-		}
 	}
 
 	parent_rel = table_open(dispatch->hypertable->main_table_relid, AccessShareLock);
@@ -709,32 +612,6 @@ ts_chunk_insert_state_create(const Chunk *chunk, ChunkDispatch *dispatch)
 
 	adjust_projections(state, dispatch, RelationGetForm(rel)->reltype);
 
-	if (has_compressed_chunk)
-	{
-		CompressChunkInsertState *compress_info = palloc0(sizeof(CompressChunkInsertState));
-		int32 htid = ts_hypertable_relid_to_id(chunk->hypertable_relid);
-		/* this is true as compressed chunks are not created on access node */
-		Assert(chunk->relkind != RELKIND_FOREIGN_TABLE);
-		Assert(compress_rel != NULL);
-		compress_info->compress_rel = compress_rel;
-		Assert(ts_cm_functions->compress_row_init != NULL);
-		/* need a way to convert from chunk tuple to compressed chunk tuple */
-		compress_info->compress_state = ts_cm_functions->compress_row_init(htid, rel, compress_rel);
-		compress_info->orig_result_relation_info = relinfo;
-		if (cagg_trig_nargs > 0) /*we found a cagg trigger earlier */
-		{
-			compress_info->has_cagg_trigger = true;
-			compress_info->cagg_trig_nargs = cagg_trig_nargs;
-			compress_info->cagg_trig_args[0] = cagg_trig_args[0];
-			compress_info->cagg_trig_args[1] = cagg_trig_args[1];
-		}
-		state->compress_info = compress_info;
-	}
-	else
-	{
-		state->compress_info = NULL;
-	}
-
 	/* Need a tuple table slot to store tuples going into this chunk. We don't
 	 * want this slot tied to the executor's tuple table, since that would tie
 	 * the slot's lifetime to the entire length of the execution and we want
@@ -751,7 +628,7 @@ ts_chunk_insert_state_create(const Chunk *chunk, ChunkDispatch *dispatch)
 	if (chunk->relkind == RELKIND_FOREIGN_TABLE)
 	{
 		RangeTblEntry *rte =
-			rt_fetch(resrelinfo->ri_RangeTableIndex, dispatch->estate->es_range_table);
+			rt_fetch(relinfo->ri_RangeTableIndex, dispatch->estate->es_range_table);
 
 		Assert(rte != NULL);
 
@@ -765,10 +642,10 @@ ts_chunk_insert_state_create(const Chunk *chunk, ChunkDispatch *dispatch)
 		 * the FDW. Instead exploit the FdwPrivate pointer to pass on the
 		 * chunk insert state to DataNodeDispatch so that it knows which data nodes
 		 * to insert into. */
-		resrelinfo->ri_FdwState = state;
+		relinfo->ri_FdwState = state;
 	}
-	else if (resrelinfo->ri_FdwRoutine && !resrelinfo->ri_usesFdwDirectModify &&
-			 resrelinfo->ri_FdwRoutine->BeginForeignModify)
+	else if (relinfo->ri_FdwRoutine && !relinfo->ri_usesFdwDirectModify &&
+			 relinfo->ri_FdwRoutine->BeginForeignModify)
 	{
 		/*
 		 * If this is a chunk located one or more data nodes, setup the
@@ -792,11 +669,11 @@ ts_chunk_insert_state_create(const Chunk *chunk, ChunkDispatch *dispatch)
 		 * tsl/src/fdw/timescaledb_fdw.c).
 		 */
 		fdwprivate = lappend(list_copy(fdwprivate), state);
-		resrelinfo->ri_FdwRoutine->BeginForeignModify(mtstate,
-													  resrelinfo,
-													  fdwprivate,
-													  0,
-													  dispatch->eflags);
+		relinfo->ri_FdwRoutine->BeginForeignModify(mtstate,
+												   relinfo,
+												   fdwprivate,
+												   0,
+												   dispatch->eflags);
 	}
 
 	MemoryContextSwitchTo(old_mcxt);
@@ -809,42 +686,19 @@ ts_chunk_insert_state_destroy(ChunkInsertState *state)
 {
 	ResultRelInfo *rri = state->result_relation_info;
 
+	if (state->chunk_compressed && !state->chunk_partial)
+	{
+		Oid chunk_relid = RelationGetRelid(state->result_relation_info->ri_RelationDesc);
+		Chunk *chunk = ts_chunk_get_by_relid(chunk_relid, true);
+		ts_chunk_set_partial(chunk);
+	}
+
 	if (rri->ri_FdwRoutine && !rri->ri_usesFdwDirectModify && rri->ri_FdwRoutine->EndForeignModify)
 		rri->ri_FdwRoutine->EndForeignModify(state->estate, rri);
 
 	destroy_on_conflict_state(state);
 	ExecCloseIndices(state->result_relation_info);
 
-	/*
-	 * The chunk search functions may leak memory, so switch to a temporary
-	 * memory context.
-	 */
-	MemoryContext old_context = MemoryContextSwitchTo(GetPerTupleMemoryContext(state->estate));
-	if (state->compress_info)
-	{
-		ResultRelInfo *orig_chunk_rri = state->compress_info->orig_result_relation_info;
-		Oid chunk_relid = RelationGetRelid(orig_chunk_rri->ri_RelationDesc);
-		ts_cm_functions->compress_row_end(state->compress_info->compress_state);
-		ts_cm_functions->compress_row_destroy(state->compress_info->compress_state);
-		Chunk *chunk = ts_chunk_get_by_relid(chunk_relid, true);
-		if (!ts_chunk_is_unordered(chunk))
-			ts_chunk_set_unordered(chunk);
-		table_close(state->compress_info->compress_rel, NoLock);
-	}
-	else if (RelationGetForm(state->result_relation_info->ri_RelationDesc)->relkind ==
-			 RELKIND_FOREIGN_TABLE)
-	{
-		/* If a distributed chunk shows compressed status on AN,
-		 * we mark it as unordered , because the insert now goes into
-		 * a previously compressed chunk
-		 */
-		Oid chunk_relid = RelationGetRelid(state->result_relation_info->ri_RelationDesc);
-		Chunk *chunk = ts_chunk_get_by_relid(chunk_relid, true);
-		if (ts_chunk_is_compressed(chunk) && (!ts_chunk_is_unordered(chunk)))
-			ts_chunk_set_unordered(chunk);
-	}
-	MemoryContextSwitchTo(old_context);
-
 	table_close(state->rel, NoLock);
 	if (state->slot)
 		ExecDropSingleTupleTableSlot(state->slot);
@@ -907,31 +761,3 @@ ts_chunk_insert_state_destroy(ChunkInsertState *state)
 	else
 		MemoryContextDelete(state->mctx);
 }
-
-/* invoke cagg trigger on a compressed chunk. AFTER Row Triggers on
- * compressed chunks do not work with the PG framework - so we explicitly
- * call the underlying C function. Note that in the case of a distr. ht, this
- * trigger is executed on the AN.
- * Parameters:
- * chunk_rel : chunk that will be modified (original chunk)
- * chunk_tuple: tuple to be inserted into the chunk (before being transformed
- *            into compressed format)
- */
-void
-ts_compress_chunk_invoke_cagg_trigger(CompressChunkInsertState *compress_info, Relation chunk_rel,
-									  HeapTuple chunk_tuple)
-{
-	Assert(ts_cm_functions->continuous_agg_call_invalidation_trigger);
-	int32 hypertable_id = compress_info->cagg_trig_args[0];
-	int32 parent_hypertable_id = compress_info->cagg_trig_args[1];
-	bool is_distributed_ht = (compress_info->cagg_trig_nargs == 2);
-	Assert((compress_info->cagg_trig_nargs == 1 && parent_hypertable_id == 0) ||
-		   (compress_info->cagg_trig_nargs == 2 && parent_hypertable_id > 0));
-	ts_cm_functions->continuous_agg_call_invalidation_trigger(hypertable_id,
-															  chunk_rel,
-															  chunk_tuple,
-															  NULL /* chunk_newtuple */,
-															  false /* update */,
-															  is_distributed_ht,
-															  parent_hypertable_id);
-}
diff --git a/src/nodes/chunk_insert_state.h b/src/nodes/chunk_insert_state.h
index e6b5e3d4b..57956241a 100644
--- a/src/nodes/chunk_insert_state.h
+++ b/src/nodes/chunk_insert_state.h
@@ -15,22 +15,6 @@
 #include "cache.h"
 #include "cross_module_fn.h"
 
-/* Info related to compressed chunk
- * Continuous aggregate triggers are called explicitly on
- * compressed chunks after INSERTS as AFTER ROW insert triggers
- * do now work with the PG infrastructure.
- * Note: the 2nd trigger arg is required for distributed hypertables.
- */
-typedef struct CompressChunkInsertState
-{
-	Relation compress_rel;					  /*compressed chunk */
-	ResultRelInfo *orig_result_relation_info; /*original chunk */
-	CompressSingleRowState *compress_state;
-	int32 cagg_trig_args[2]; /* cagg trigger args are hypertable ids */
-	bool has_cagg_trigger;
-	int cagg_trig_nargs;
-} CompressChunkInsertState;
-
 typedef struct TSCopyMultiInsertBuffer TSCopyMultiInsertBuffer;
 
 typedef struct ChunkInsertState
@@ -74,14 +58,13 @@ typedef struct ChunkInsertState
 	Oid user_id;
 
 	/* for tracking compressed chunks */
-	CompressChunkInsertState *compress_info;
+	bool chunk_compressed;
+	bool chunk_partial;
 } ChunkInsertState;
 
 typedef struct ChunkDispatch ChunkDispatch;
 
 extern ChunkInsertState *ts_chunk_insert_state_create(const Chunk *chunk, ChunkDispatch *dispatch);
 extern void ts_chunk_insert_state_destroy(ChunkInsertState *state);
-extern void ts_compress_chunk_invoke_cagg_trigger(CompressChunkInsertState *compress_info,
-												  Relation chunk_rel, HeapTuple tuple);
 
 #endif /* TIMESCALEDB_CHUNK_INSERT_STATE_H */
diff --git a/src/planner/planner.c b/src/planner/planner.c
index a89759e29..fcda43cf2 100644
--- a/src/planner/planner.c
+++ b/src/planner/planner.c
@@ -864,10 +864,31 @@ should_chunk_append(Hypertable *ht, PlannerInfo *root, RelOptInfo *rel, Path *pa
 			{
 				MergeAppendPath *merge = castNode(MergeAppendPath, path);
 				PathKey *pk;
+				ListCell *lc;
 
 				if (!ordered || path->pathkeys == NIL || list_length(merge->subpaths) == 0)
 					return false;
 
+				/*
+				 * Check for partial compressed chunks.
+				 *
+				 * When partial compressed chunks are present we can not do 1-level
+				 * ordered append. We instead need nested Appends to correctly preserve
+				 * ordering. For now we skip ordered append optimization when we encounter
+				 * partial chunks.
+				 */
+				foreach (lc, merge->subpaths)
+				{
+					Path *child = lfirst(lc);
+					RelOptInfo *chunk_rel = child->parent;
+					if (chunk_rel->fdw_private)
+					{
+						TimescaleDBPrivate *private = chunk_rel->fdw_private;
+						if (private->chunk && ts_chunk_is_partial(private->chunk))
+							return false;
+					}
+				}
+
 				/* cannot support ordered append with OSM chunks. OSM chunk
 				 * ranges are not recorded with the catalog
 				 */
diff --git a/tsl/src/nodes/decompress_chunk/decompress_chunk.c b/tsl/src/nodes/decompress_chunk/decompress_chunk.c
index a7778824b..d331c520e 100644
--- a/tsl/src/nodes/decompress_chunk/decompress_chunk.c
+++ b/tsl/src/nodes/decompress_chunk/decompress_chunk.c
@@ -21,6 +21,7 @@
 
 #include <planner.h>
 
+#include "compat/compat.h"
 #include "ts_catalog/hypertable_compression.h"
 #include "import/planner.h"
 #include "compression/create.h"
@@ -384,6 +385,10 @@ ts_decompress_chunk_generate_paths(PlannerInfo *root, RelOptInfo *chunk_rel, Hyp
 
 	Assert(chunk->fd.compressed_chunk_id > 0);
 
+	Path *uncompressed_path = chunk_rel->pathlist ? (Path *) linitial(chunk_rel->pathlist) : NULL;
+	Path *uncompressed_partial_path =
+		chunk_rel->partial_pathlist ? (Path *) linitial(chunk_rel->partial_pathlist) : NULL;
+	Assert(uncompressed_path);
 	chunk_rel->pathlist = NIL;
 	chunk_rel->partial_pathlist = NIL;
 
@@ -393,7 +398,11 @@ ts_decompress_chunk_generate_paths(PlannerInfo *root, RelOptInfo *chunk_rel, Hyp
 
 	compressed_rel->consider_parallel = chunk_rel->consider_parallel;
 	/* translate chunk_rel->baserestrictinfo */
-	pushdown_quals(root, chunk_rel, compressed_rel, info->hypertable_compression_info);
+	pushdown_quals(root,
+				   chunk_rel,
+				   compressed_rel,
+				   info->hypertable_compression_info,
+				   ts_chunk_is_partial(chunk));
 	set_baserel_size_estimates(root, compressed_rel);
 	new_row_estimate = compressed_rel->rows * DECOMPRESS_CHUNK_BATCH_SIZE;
 
@@ -423,7 +432,7 @@ ts_decompress_chunk_generate_paths(PlannerInfo *root, RelOptInfo *chunk_rel, Hyp
 	foreach (lc, compressed_rel->pathlist)
 	{
 		Path *child_path = lfirst(lc);
-		DecompressChunkPath *path;
+		Path *path;
 
 		/*
 		 * We skip any BitmapScan parameterized paths here as supporting
@@ -503,7 +512,7 @@ ts_decompress_chunk_generate_paths(PlannerInfo *root, RelOptInfo *chunk_rel, Hyp
 				continue;
 		}
 
-		path = decompress_chunk_path_create(root, info, 0, child_path);
+		path = (Path *) decompress_chunk_path_create(root, info, 0, child_path);
 
 		/* If we can push down the sort below the DecompressChunk node, we set the pathkeys of the
 		 * decompress node to the query pathkeys, while remembering the compressed_pathkeys
@@ -540,9 +549,24 @@ ts_decompress_chunk_generate_paths(PlannerInfo *root, RelOptInfo *chunk_rel, Hyp
 			add_path(chunk_rel, &dcpath->cpath.path);
 		}
 
+		/*
+		 * If this is a partially compressed chunk we have to combine data
+		 * from compressed and uncompressed chunk.
+		 */
+		if (ts_chunk_is_partial(chunk))
+			path = (Path *) create_append_path_compat(root,
+													  chunk_rel,
+													  list_make2(path, uncompressed_path),
+													  NIL /* partial paths */,
+													  NIL /* pathkeys */,
+													  PATH_REQ_OUTER(uncompressed_path),
+													  0,
+													  false,
+													  false,
+													  path->rows + uncompressed_path->rows);
 		/* this has to go after the path is copied for the ordered path since path can get freed in
 		 * add_path */
-		add_path(chunk_rel, &path->cpath.path);
+		add_path(chunk_rel, path);
 	}
 	/* the chunk_rel now owns the paths, remove them from the compressed_rel so they can't be freed
 	 * if it's planned */
@@ -553,14 +577,31 @@ ts_decompress_chunk_generate_paths(PlannerInfo *root, RelOptInfo *chunk_rel, Hyp
 		foreach (lc, compressed_rel->partial_pathlist)
 		{
 			Path *child_path = lfirst(lc);
-			DecompressChunkPath *path;
+			Path *path;
 			if (child_path->param_info != NULL &&
 				(bms_is_member(chunk_rel->relid, child_path->param_info->ppi_req_outer) ||
 				 (!info->single_chunk &&
 				  bms_is_member(ht_relid, child_path->param_info->ppi_req_outer))))
 				continue;
-			path = decompress_chunk_path_create(root, info, parallel_workers, child_path);
-			add_partial_path(chunk_rel, &path->cpath.path);
+
+			/*
+			 * If this is a partially compressed chunk we have to combine data
+			 * from compressed and uncompressed chunk.
+			 */
+			path = (Path *) decompress_chunk_path_create(root, info, parallel_workers, child_path);
+			if (ts_chunk_is_partial(chunk))
+				path =
+					(Path *) create_append_path_compat(root,
+													   chunk_rel,
+													   NIL,
+													   list_make2(path, uncompressed_partial_path),
+													   NIL /* pathkeys */,
+													   PATH_REQ_OUTER(uncompressed_partial_path),
+													   parallel_workers,
+													   false,
+													   NIL,
+													   path->rows + uncompressed_path->rows);
+			add_partial_path(chunk_rel, path);
 		}
 		/* the chunk_rel now owns the paths, remove them from the compressed_rel so they can't be
 		 * freed if it's planned */
@@ -1394,7 +1435,7 @@ build_sortinfo(Chunk *chunk, RelOptInfo *chunk_rel, CompressionInfo *info, List
 	ListCell *lc = list_head(pathkeys);
 	SortInfo sort_info = { .can_pushdown_sort = false, .needs_sequence_num = false };
 
-	if (pathkeys == NIL || ts_chunk_is_unordered(chunk))
+	if (pathkeys == NIL || ts_chunk_is_unordered(chunk) || ts_chunk_is_partial(chunk))
 		return sort_info;
 
 	/* all segmentby columns need to be prefix of pathkeys */
diff --git a/tsl/src/nodes/decompress_chunk/qual_pushdown.c b/tsl/src/nodes/decompress_chunk/qual_pushdown.c
index 9bf677e37..69b3697a6 100644
--- a/tsl/src/nodes/decompress_chunk/qual_pushdown.c
+++ b/tsl/src/nodes/decompress_chunk/qual_pushdown.c
@@ -35,7 +35,7 @@ static Node *modify_expression(Node *node, QualPushdownContext *context);
 
 void
 pushdown_quals(PlannerInfo *root, RelOptInfo *chunk_rel, RelOptInfo *compressed_rel,
-			   List *compression_info)
+			   List *compression_info, bool chunk_partial)
 {
 	ListCell *lc;
 	List *decompress_clauses = NIL;
@@ -83,7 +83,7 @@ pushdown_quals(PlannerInfo *root, RelOptInfo *chunk_rel, RelOptInfo *compressed_
 		}
 		/* We need to check the restriction clause on the decompress node if the clause can't be
 		 * pushed down or needs re-checking */
-		if (!context.can_pushdown || context.needs_recheck)
+		if (!context.can_pushdown || context.needs_recheck || chunk_partial)
 		{
 			decompress_clauses = lappend(decompress_clauses, ri);
 		}
diff --git a/tsl/src/nodes/decompress_chunk/qual_pushdown.h b/tsl/src/nodes/decompress_chunk/qual_pushdown.h
index 8a9e926bf..a5f59d6fa 100644
--- a/tsl/src/nodes/decompress_chunk/qual_pushdown.h
+++ b/tsl/src/nodes/decompress_chunk/qual_pushdown.h
@@ -7,4 +7,4 @@
 #include <postgres.h>
 
 void pushdown_quals(PlannerInfo *root, RelOptInfo *chunk_rel, RelOptInfo *compressed_rel,
-					List *compression_info);
+					List *compression_info, bool chunk_partial);
diff --git a/tsl/src/remote/dist_copy.c b/tsl/src/remote/dist_copy.c
index 40c3c9459..dfe3a065f 100644
--- a/tsl/src/remote/dist_copy.c
+++ b/tsl/src/remote/dist_copy.c
@@ -1392,11 +1392,11 @@ remote_copy_process_and_send_data(RemoteCopyContext *context)
 
 		/*
 		 * For remote copy, we don't use chunk insert states on the AN.
-		 * So we need to explicitly set the chunk as unordered when copies
+		 * So we need to explicitly set the chunk as partial when copies
 		 * are directed to previously compressed chunks.
 		 */
-		if (ts_chunk_is_compressed(chunk) && (!ts_chunk_is_unordered(chunk)))
-			ts_chunk_set_unordered(chunk);
+		if (ts_chunk_is_compressed(chunk) && (!ts_chunk_is_partial(chunk)))
+			ts_chunk_set_partial(chunk);
 
 		/*
 		 * Schedule the row for sending to the data nodes containing the chunk.
diff --git a/tsl/test/expected/bgw_custom.out b/tsl/test/expected/bgw_custom.out
index 1b9649299..cb0258490 100644
--- a/tsl/test/expected/bgw_custom.out
+++ b/tsl/test/expected/bgw_custom.out
@@ -466,9 +466,9 @@ where hypertable_id = (select id from _timescaledb_catalog.hypertable
 order by id;
  id |    table_name    | status 
 ----+------------------+--------
-  1 | _hyper_1_1_chunk |      3
-  2 | _hyper_1_2_chunk |      3
-  3 | _hyper_1_3_chunk |      3
+  1 | _hyper_1_1_chunk |      9
+  2 | _hyper_1_2_chunk |      9
+  3 | _hyper_1_3_chunk |      9
 (3 rows)
 
 --running job second time, wait for it to complete
diff --git a/tsl/test/expected/compression_bgw.out b/tsl/test/expected/compression_bgw.out
index da35b2ee7..593d5d80f 100644
--- a/tsl/test/expected/compression_bgw.out
+++ b/tsl/test/expected/compression_bgw.out
@@ -417,7 +417,7 @@ FROM compressed_chunk_info_view
 WHERE hypertable_name = 'test2' ORDER BY chunk_name;
  chunk_status |     CHUNK_NAME     
 --------------+--------------------
-            3 | _hyper_14_62_chunk
+            9 | _hyper_14_62_chunk
 (1 row)
 
 SELECT compressed_chunk_schema || '.' || compressed_chunk_name as "COMP_CHUNK_NAME",
@@ -482,7 +482,7 @@ FROM compressed_chunk_info_view
 WHERE hypertable_name = 'test2' ORDER BY chunk_name;
  chunk_status |     CHUNK_NAME     
 --------------+--------------------
-            3 | _hyper_14_62_chunk
+            9 | _hyper_14_62_chunk
             0 | _hyper_14_65_chunk
 (2 rows)
 
@@ -543,7 +543,7 @@ INSERT INTO metrics SELECT '2000-01-01';
 SELECT chunk_status FROM compressed_chunk_info_view WHERE hypertable_name = 'metrics';
  chunk_status 
 --------------
-            3
+            9
 (1 row)
 
 -- should recompress
@@ -577,7 +577,7 @@ INSERT INTO metrics SELECT '2000-01-01';
 SELECT chunk_status FROM compressed_chunk_info_view WHERE hypertable_name = 'metrics';
  chunk_status 
 --------------
-            3
+            9
 (1 row)
 
 -- still nothing to do since we disabled recompress
@@ -586,7 +586,7 @@ CALL run_job(:JOB_COMPRESS);
 SELECT chunk_status FROM compressed_chunk_info_view WHERE hypertable_name = 'metrics';
  chunk_status 
 --------------
-            3
+            9
 (1 row)
 
 -- reenable recompress in compress job
@@ -635,7 +635,7 @@ INSERT INTO metrics SELECT '2000-01-01';
 SELECT chunk_status FROM compressed_chunk_info_view WHERE hypertable_name = 'metrics';
  chunk_status 
 --------------
-            3
+            9
 (1 row)
 
 CALL run_job(:JOB_RECOMPRESS);
@@ -643,7 +643,7 @@ CALL run_job(:JOB_RECOMPRESS);
 SELECT chunk_status FROM compressed_chunk_info_view WHERE hypertable_name = 'metrics';
  chunk_status 
 --------------
-            1
+            9
 (1 row)
 
 SELECT delete_job(:JOB_RECOMPRESS);
diff --git a/tsl/test/expected/compression_insert-12.out b/tsl/test/expected/compression_insert-12.out
index 974bff4ca..b0ee216ad 100644
--- a/tsl/test/expected/compression_insert-12.out
+++ b/tsl/test/expected/compression_insert-12.out
@@ -40,7 +40,7 @@ INSERT INTO test1 SELECT '2020-01-02 11:16:00-05' , 11, 16, 'new' ;
 SELECT COUNT(*) from _timescaledb_internal.compress_hyper_2_2_chunk;
  count 
 -------
-     2
+     1
 (1 row)
 
 INSERT INTO test1 SELECT '2020-01-02 11:16:00-05' , i, i +5, 'clay'
@@ -48,7 +48,7 @@ FROM (Select generate_series(10, 20, 1) i ) q;
 SELECT COUNT(*) from _timescaledb_internal.compress_hyper_2_2_chunk;
  count 
 -------
-    13
+     1
 (1 row)
 
 SELECT count(*) from test1;
@@ -62,7 +62,7 @@ COPY test1 FROM STDIN DELIMITER ',';
 SELECT COUNT(*) from _timescaledb_internal.compress_hyper_2_2_chunk;
  count 
 -------
-    14
+     1
 (1 row)
 
 -- multi row copy
@@ -70,7 +70,7 @@ COPY test1 FROM STDIN DELIMITER ',';
 SELECT COUNT(*) from _timescaledb_internal.compress_hyper_2_2_chunk;
  count 
 -------
-    17
+     1
 (1 row)
 
 --Verify that all the data went into the initial chunk
@@ -151,7 +151,7 @@ ALTER TABLE test1 ADD COLUMN newtcol varchar(400);
 SELECT count(*) from _timescaledb_internal.compress_hyper_2_2_chunk;
  count 
 -------
-    19
+     1
 (1 row)
 
 INSERT INTO test1 SELECT '2020-01-02 11:16:00-05' , 100, 101, 'prev101', 'this is the newtcol101';
@@ -176,7 +176,7 @@ SELECT * FROM test1 WHERE b = 16 order by 1, 2, 3, 4, 5;
 SELECT count(*) from _timescaledb_internal.compress_hyper_2_2_chunk;
  count 
 -------
-    26
+     1
 (1 row)
 
 SELECT count(*)
@@ -192,7 +192,7 @@ COPY test1 FROM STDIN DELIMITER ',';
 SELECT count(*) from _timescaledb_internal.compress_hyper_2_2_chunk;
  count 
 -------
-    30
+     1
 (1 row)
 
 SELECT count(*)
@@ -665,21 +665,16 @@ CREATE TRIGGER t4_ar AFTER INSERT ON trigger_test FOR EACH ROW EXECUTE FUNCTION
 \set ON_ERROR_STOP 0
 \set VERBOSITY default
 INSERT INTO trigger_test SELECT '2000-01-01',1,0;
-ERROR:  after insert row trigger on compressed chunk not supported
-DETAIL:  Triggers: t4_ar
-HINT:  Decompress the chunk first before inserting into it.
+NOTICE:  Trigger t4_ar AFTER INSERT ROW on _hyper_11_15_chunk: ("Sat Jan 01 00:00:00 2000 PST",1,0) <NULL>
 COPY trigger_test FROM STDIN DELIMITER ',';
-ERROR:  after insert row trigger on compressed chunk not supported
-DETAIL:  Triggers: t4_ar
-HINT:  Decompress the chunk first before inserting into it.
-CONTEXT:  COPY trigger_test, line 1: "2000-01-01 01:00:00-05,1,0"
+NOTICE:  Trigger t4_ar AFTER INSERT ROW on _hyper_11_15_chunk: ("Fri Dec 31 22:00:00 1999 PST",1,0) <NULL>
 \set VERBOSITY terse
 \set ON_ERROR_STOP 1
 -- should not insert rows. count is 1
 SELECT count(*) FROM trigger_test;
  count 
 -------
-     1
+     3
 (1 row)
 
 BEGIN;
@@ -691,7 +686,7 @@ NOTICE:  Trigger t4_ar AFTER INSERT ROW on _hyper_11_17_chunk: ("Sun Dec 31 22:0
 SELECT count(*) FROM trigger_test;
  count 
 -------
-     3
+     5
 (1 row)
 
 ROLLBACK;
@@ -700,15 +695,15 @@ DROP TRIGGER t4_ar ON trigger_test;
 CREATE CONSTRAINT TRIGGER t4_constraint AFTER INSERT ON trigger_test FOR EACH ROW EXECUTE FUNCTION stmt_trig_info();
 \set ON_ERROR_STOP 0
 INSERT INTO trigger_test SELECT '2000-01-01',1,0;
-ERROR:  after insert row trigger on compressed chunk not supported
+NOTICE:  Trigger t4_constraint AFTER INSERT ROW on _hyper_11_15_chunk: ("Sat Jan 01 00:00:00 2000 PST",1,0) <NULL>
 COPY trigger_test FROM STDIN DELIMITER ',';
-ERROR:  after insert row trigger on compressed chunk not supported
+NOTICE:  Trigger t4_constraint AFTER INSERT ROW on _hyper_11_15_chunk: ("Fri Dec 31 22:00:00 1999 PST",1,0) <NULL>
 \set ON_ERROR_STOP 1
 -- should not insert rows. count is 1
 SELECT count(*) FROM trigger_test;
  count 
 -------
-     1
+     5
 (1 row)
 
 DROP trigger t4_constraint ON trigger_test;
@@ -730,7 +725,7 @@ CREATE TRIGGER t1_mod BEFORE INSERT ON trigger_test FOR EACH ROW EXECUTE FUNCTIO
 SELECT count(*) FROM trigger_test;
  count 
 -------
-     2
+     6
 (1 row)
 
 BEGIN;
@@ -742,11 +737,15 @@ NOTICE:  Trigger t1_mod BEFORE INSERT ROW on _hyper_11_18_chunk: ("Fri Jan 01 00
 SELECT * FROM trigger_test ORDER BY 1 ,2, 5;
              time             | device | value | addcolv | addcoli 
 ------------------------------+--------+-------+---------+---------
+ Fri Dec 31 22:00:00 1999 PST |      1 |     0 |         |        
+ Fri Dec 31 22:00:00 1999 PST |      1 |     0 |         |        
  Sat Jan 01 00:00:00 2000 PST |      1 |   111 | eleven  |     111
  Sat Jan 01 00:00:00 2000 PST |      1 |     1 |         |        
+ Sat Jan 01 00:00:00 2000 PST |      1 |     0 |         |        
+ Sat Jan 01 00:00:00 2000 PST |      1 |     0 |         |        
  Fri Jan 01 00:00:00 2010 PST |     10 |    10 | ten     |     222
  Fri Jan 01 00:00:00 2010 PST |     10 |   110 | ten     |     222
-(4 rows)
+(8 rows)
 
 ROLLBACK;
 DROP TABLE trigger_test;
@@ -791,27 +790,32 @@ SELECT compress_chunk(format('%I.%I',chunk_schema,chunk_name), true) FROM timesc
 INSERT INTO test_ordering SELECT 1;
 -- should not be ordered append
 :PREFIX SELECT * FROM test_ordering ORDER BY 1;
-                        QUERY PLAN                         
------------------------------------------------------------
+                           QUERY PLAN                            
+-----------------------------------------------------------------
  Sort
    Sort Key: _hyper_13_20_chunk."time"
-   ->  Custom Scan (DecompressChunk) on _hyper_13_20_chunk
-         ->  Seq Scan on compress_hyper_14_21_chunk
-(4 rows)
+   ->  Append
+         ->  Custom Scan (DecompressChunk) on _hyper_13_20_chunk
+               ->  Seq Scan on compress_hyper_14_21_chunk
+         ->  Seq Scan on _hyper_13_20_chunk
+(6 rows)
 
 INSERT INTO test_ordering VALUES (105),(104),(103);
 -- should be ordered append
 :PREFIX SELECT * FROM test_ordering ORDER BY 1;
                                               QUERY PLAN                                              
 ------------------------------------------------------------------------------------------------------
- Custom Scan (ChunkAppend) on test_ordering
-   Order: test_ordering."time"
+ Merge Append
+   Sort Key: _hyper_13_20_chunk."time"
    ->  Sort
          Sort Key: _hyper_13_20_chunk."time"
          ->  Custom Scan (DecompressChunk) on _hyper_13_20_chunk
                ->  Seq Scan on compress_hyper_14_21_chunk
+   ->  Sort
+         Sort Key: _hyper_13_20_chunk."time"
+         ->  Seq Scan on _hyper_13_20_chunk
    ->  Index Only Scan Backward using _hyper_13_22_chunk_test_ordering_time_idx on _hyper_13_22_chunk
-(7 rows)
+(10 rows)
 
 --insert into compressed + uncompressed chunk
 INSERT INTO test_ordering VALUES (21), (22),(113);
@@ -844,17 +848,20 @@ NOTICE:  chunk "_hyper_13_20_chunk" is already compressed
 :PREFIX SELECT * FROM test_ordering ORDER BY 1;
                                   QUERY PLAN                                   
 -------------------------------------------------------------------------------
- Custom Scan (ChunkAppend) on test_ordering
-   Order: test_ordering."time"
+ Merge Append
+   Sort Key: _hyper_13_20_chunk."time"
    ->  Sort
          Sort Key: _hyper_13_20_chunk."time"
          ->  Custom Scan (DecompressChunk) on _hyper_13_20_chunk
                ->  Seq Scan on compress_hyper_14_21_chunk
+   ->  Sort
+         Sort Key: _hyper_13_20_chunk."time"
+         ->  Seq Scan on _hyper_13_20_chunk
    ->  Custom Scan (DecompressChunk) on _hyper_13_22_chunk
          ->  Sort
                Sort Key: compress_hyper_14_23_chunk._ts_meta_sequence_num DESC
                ->  Seq Scan on compress_hyper_14_23_chunk
-(10 rows)
+(13 rows)
 
 -- TEST cagg triggers with insert into compressed chunk
 CREATE TABLE conditions (
diff --git a/tsl/test/expected/compression_insert-13.out b/tsl/test/expected/compression_insert-13.out
index 2b2b00f89..8e2ee9321 100644
--- a/tsl/test/expected/compression_insert-13.out
+++ b/tsl/test/expected/compression_insert-13.out
@@ -40,7 +40,7 @@ INSERT INTO test1 SELECT '2020-01-02 11:16:00-05' , 11, 16, 'new' ;
 SELECT COUNT(*) from _timescaledb_internal.compress_hyper_2_2_chunk;
  count 
 -------
-     2
+     1
 (1 row)
 
 INSERT INTO test1 SELECT '2020-01-02 11:16:00-05' , i, i +5, 'clay'
@@ -48,7 +48,7 @@ FROM (Select generate_series(10, 20, 1) i ) q;
 SELECT COUNT(*) from _timescaledb_internal.compress_hyper_2_2_chunk;
  count 
 -------
-    13
+     1
 (1 row)
 
 SELECT count(*) from test1;
@@ -62,7 +62,7 @@ COPY test1 FROM STDIN DELIMITER ',';
 SELECT COUNT(*) from _timescaledb_internal.compress_hyper_2_2_chunk;
  count 
 -------
-    14
+     1
 (1 row)
 
 -- multi row copy
@@ -70,7 +70,7 @@ COPY test1 FROM STDIN DELIMITER ',';
 SELECT COUNT(*) from _timescaledb_internal.compress_hyper_2_2_chunk;
  count 
 -------
-    17
+     1
 (1 row)
 
 --Verify that all the data went into the initial chunk
@@ -151,7 +151,7 @@ ALTER TABLE test1 ADD COLUMN newtcol varchar(400);
 SELECT count(*) from _timescaledb_internal.compress_hyper_2_2_chunk;
  count 
 -------
-    19
+     1
 (1 row)
 
 INSERT INTO test1 SELECT '2020-01-02 11:16:00-05' , 100, 101, 'prev101', 'this is the newtcol101';
@@ -176,7 +176,7 @@ SELECT * FROM test1 WHERE b = 16 order by 1, 2, 3, 4, 5;
 SELECT count(*) from _timescaledb_internal.compress_hyper_2_2_chunk;
  count 
 -------
-    26
+     1
 (1 row)
 
 SELECT count(*)
@@ -192,7 +192,7 @@ COPY test1 FROM STDIN DELIMITER ',';
 SELECT count(*) from _timescaledb_internal.compress_hyper_2_2_chunk;
  count 
 -------
-    30
+     1
 (1 row)
 
 SELECT count(*)
@@ -665,21 +665,16 @@ CREATE TRIGGER t4_ar AFTER INSERT ON trigger_test FOR EACH ROW EXECUTE FUNCTION
 \set ON_ERROR_STOP 0
 \set VERBOSITY default
 INSERT INTO trigger_test SELECT '2000-01-01',1,0;
-ERROR:  after insert row trigger on compressed chunk not supported
-DETAIL:  Triggers: t4_ar
-HINT:  Decompress the chunk first before inserting into it.
+NOTICE:  Trigger t4_ar AFTER INSERT ROW on _hyper_11_15_chunk: ("Sat Jan 01 00:00:00 2000 PST",1,0) <NULL>
 COPY trigger_test FROM STDIN DELIMITER ',';
-ERROR:  after insert row trigger on compressed chunk not supported
-DETAIL:  Triggers: t4_ar
-HINT:  Decompress the chunk first before inserting into it.
-CONTEXT:  COPY trigger_test, line 1: "2000-01-01 01:00:00-05,1,0"
+NOTICE:  Trigger t4_ar AFTER INSERT ROW on _hyper_11_15_chunk: ("Fri Dec 31 22:00:00 1999 PST",1,0) <NULL>
 \set VERBOSITY terse
 \set ON_ERROR_STOP 1
 -- should not insert rows. count is 1
 SELECT count(*) FROM trigger_test;
  count 
 -------
-     1
+     3
 (1 row)
 
 BEGIN;
@@ -691,7 +686,7 @@ NOTICE:  Trigger t4_ar AFTER INSERT ROW on _hyper_11_17_chunk: ("Sun Dec 31 22:0
 SELECT count(*) FROM trigger_test;
  count 
 -------
-     3
+     5
 (1 row)
 
 ROLLBACK;
@@ -700,15 +695,15 @@ DROP TRIGGER t4_ar ON trigger_test;
 CREATE CONSTRAINT TRIGGER t4_constraint AFTER INSERT ON trigger_test FOR EACH ROW EXECUTE FUNCTION stmt_trig_info();
 \set ON_ERROR_STOP 0
 INSERT INTO trigger_test SELECT '2000-01-01',1,0;
-ERROR:  after insert row trigger on compressed chunk not supported
+NOTICE:  Trigger t4_constraint AFTER INSERT ROW on _hyper_11_15_chunk: ("Sat Jan 01 00:00:00 2000 PST",1,0) <NULL>
 COPY trigger_test FROM STDIN DELIMITER ',';
-ERROR:  after insert row trigger on compressed chunk not supported
+NOTICE:  Trigger t4_constraint AFTER INSERT ROW on _hyper_11_15_chunk: ("Fri Dec 31 22:00:00 1999 PST",1,0) <NULL>
 \set ON_ERROR_STOP 1
 -- should not insert rows. count is 1
 SELECT count(*) FROM trigger_test;
  count 
 -------
-     1
+     5
 (1 row)
 
 DROP trigger t4_constraint ON trigger_test;
@@ -730,7 +725,7 @@ CREATE TRIGGER t1_mod BEFORE INSERT ON trigger_test FOR EACH ROW EXECUTE FUNCTIO
 SELECT count(*) FROM trigger_test;
  count 
 -------
-     2
+     6
 (1 row)
 
 BEGIN;
@@ -742,11 +737,15 @@ NOTICE:  Trigger t1_mod BEFORE INSERT ROW on _hyper_11_18_chunk: ("Fri Jan 01 00
 SELECT * FROM trigger_test ORDER BY 1 ,2, 5;
              time             | device | value | addcolv | addcoli 
 ------------------------------+--------+-------+---------+---------
+ Fri Dec 31 22:00:00 1999 PST |      1 |     0 |         |        
+ Fri Dec 31 22:00:00 1999 PST |      1 |     0 |         |        
  Sat Jan 01 00:00:00 2000 PST |      1 |   111 | eleven  |     111
  Sat Jan 01 00:00:00 2000 PST |      1 |     1 |         |        
+ Sat Jan 01 00:00:00 2000 PST |      1 |     0 |         |        
+ Sat Jan 01 00:00:00 2000 PST |      1 |     0 |         |        
  Fri Jan 01 00:00:00 2010 PST |     10 |    10 | ten     |     222
  Fri Jan 01 00:00:00 2010 PST |     10 |   110 | ten     |     222
-(4 rows)
+(8 rows)
 
 ROLLBACK;
 DROP TABLE trigger_test;
@@ -791,27 +790,32 @@ SELECT compress_chunk(format('%I.%I',chunk_schema,chunk_name), true) FROM timesc
 INSERT INTO test_ordering SELECT 1;
 -- should not be ordered append
 :PREFIX SELECT * FROM test_ordering ORDER BY 1;
-                        QUERY PLAN                         
------------------------------------------------------------
+                           QUERY PLAN                            
+-----------------------------------------------------------------
  Sort
    Sort Key: _hyper_13_20_chunk."time"
-   ->  Custom Scan (DecompressChunk) on _hyper_13_20_chunk
-         ->  Seq Scan on compress_hyper_14_21_chunk
-(4 rows)
+   ->  Append
+         ->  Custom Scan (DecompressChunk) on _hyper_13_20_chunk
+               ->  Seq Scan on compress_hyper_14_21_chunk
+         ->  Seq Scan on _hyper_13_20_chunk
+(6 rows)
 
 INSERT INTO test_ordering VALUES (105),(104),(103);
 -- should be ordered append
 :PREFIX SELECT * FROM test_ordering ORDER BY 1;
                                               QUERY PLAN                                              
 ------------------------------------------------------------------------------------------------------
- Custom Scan (ChunkAppend) on test_ordering
-   Order: test_ordering."time"
+ Merge Append
+   Sort Key: _hyper_13_20_chunk."time"
    ->  Sort
          Sort Key: _hyper_13_20_chunk."time"
          ->  Custom Scan (DecompressChunk) on _hyper_13_20_chunk
                ->  Seq Scan on compress_hyper_14_21_chunk
+   ->  Sort
+         Sort Key: _hyper_13_20_chunk."time"
+         ->  Seq Scan on _hyper_13_20_chunk
    ->  Index Only Scan Backward using _hyper_13_22_chunk_test_ordering_time_idx on _hyper_13_22_chunk
-(7 rows)
+(10 rows)
 
 --insert into compressed + uncompressed chunk
 INSERT INTO test_ordering VALUES (21), (22),(113);
@@ -844,17 +848,20 @@ NOTICE:  chunk "_hyper_13_20_chunk" is already compressed
 :PREFIX SELECT * FROM test_ordering ORDER BY 1;
                                   QUERY PLAN                                   
 -------------------------------------------------------------------------------
- Custom Scan (ChunkAppend) on test_ordering
-   Order: test_ordering."time"
+ Merge Append
+   Sort Key: _hyper_13_20_chunk."time"
    ->  Sort
          Sort Key: _hyper_13_20_chunk."time"
          ->  Custom Scan (DecompressChunk) on _hyper_13_20_chunk
                ->  Seq Scan on compress_hyper_14_21_chunk
+   ->  Sort
+         Sort Key: _hyper_13_20_chunk."time"
+         ->  Seq Scan on _hyper_13_20_chunk
    ->  Custom Scan (DecompressChunk) on _hyper_13_22_chunk
          ->  Sort
                Sort Key: compress_hyper_14_23_chunk._ts_meta_sequence_num DESC
                ->  Seq Scan on compress_hyper_14_23_chunk
-(10 rows)
+(13 rows)
 
 -- TEST cagg triggers with insert into compressed chunk
 CREATE TABLE conditions (
diff --git a/tsl/test/expected/compression_insert-14.out b/tsl/test/expected/compression_insert-14.out
index 2b2b00f89..8e2ee9321 100644
--- a/tsl/test/expected/compression_insert-14.out
+++ b/tsl/test/expected/compression_insert-14.out
@@ -40,7 +40,7 @@ INSERT INTO test1 SELECT '2020-01-02 11:16:00-05' , 11, 16, 'new' ;
 SELECT COUNT(*) from _timescaledb_internal.compress_hyper_2_2_chunk;
  count 
 -------
-     2
+     1
 (1 row)
 
 INSERT INTO test1 SELECT '2020-01-02 11:16:00-05' , i, i +5, 'clay'
@@ -48,7 +48,7 @@ FROM (Select generate_series(10, 20, 1) i ) q;
 SELECT COUNT(*) from _timescaledb_internal.compress_hyper_2_2_chunk;
  count 
 -------
-    13
+     1
 (1 row)
 
 SELECT count(*) from test1;
@@ -62,7 +62,7 @@ COPY test1 FROM STDIN DELIMITER ',';
 SELECT COUNT(*) from _timescaledb_internal.compress_hyper_2_2_chunk;
  count 
 -------
-    14
+     1
 (1 row)
 
 -- multi row copy
@@ -70,7 +70,7 @@ COPY test1 FROM STDIN DELIMITER ',';
 SELECT COUNT(*) from _timescaledb_internal.compress_hyper_2_2_chunk;
  count 
 -------
-    17
+     1
 (1 row)
 
 --Verify that all the data went into the initial chunk
@@ -151,7 +151,7 @@ ALTER TABLE test1 ADD COLUMN newtcol varchar(400);
 SELECT count(*) from _timescaledb_internal.compress_hyper_2_2_chunk;
  count 
 -------
-    19
+     1
 (1 row)
 
 INSERT INTO test1 SELECT '2020-01-02 11:16:00-05' , 100, 101, 'prev101', 'this is the newtcol101';
@@ -176,7 +176,7 @@ SELECT * FROM test1 WHERE b = 16 order by 1, 2, 3, 4, 5;
 SELECT count(*) from _timescaledb_internal.compress_hyper_2_2_chunk;
  count 
 -------
-    26
+     1
 (1 row)
 
 SELECT count(*)
@@ -192,7 +192,7 @@ COPY test1 FROM STDIN DELIMITER ',';
 SELECT count(*) from _timescaledb_internal.compress_hyper_2_2_chunk;
  count 
 -------
-    30
+     1
 (1 row)
 
 SELECT count(*)
@@ -665,21 +665,16 @@ CREATE TRIGGER t4_ar AFTER INSERT ON trigger_test FOR EACH ROW EXECUTE FUNCTION
 \set ON_ERROR_STOP 0
 \set VERBOSITY default
 INSERT INTO trigger_test SELECT '2000-01-01',1,0;
-ERROR:  after insert row trigger on compressed chunk not supported
-DETAIL:  Triggers: t4_ar
-HINT:  Decompress the chunk first before inserting into it.
+NOTICE:  Trigger t4_ar AFTER INSERT ROW on _hyper_11_15_chunk: ("Sat Jan 01 00:00:00 2000 PST",1,0) <NULL>
 COPY trigger_test FROM STDIN DELIMITER ',';
-ERROR:  after insert row trigger on compressed chunk not supported
-DETAIL:  Triggers: t4_ar
-HINT:  Decompress the chunk first before inserting into it.
-CONTEXT:  COPY trigger_test, line 1: "2000-01-01 01:00:00-05,1,0"
+NOTICE:  Trigger t4_ar AFTER INSERT ROW on _hyper_11_15_chunk: ("Fri Dec 31 22:00:00 1999 PST",1,0) <NULL>
 \set VERBOSITY terse
 \set ON_ERROR_STOP 1
 -- should not insert rows. count is 1
 SELECT count(*) FROM trigger_test;
  count 
 -------
-     1
+     3
 (1 row)
 
 BEGIN;
@@ -691,7 +686,7 @@ NOTICE:  Trigger t4_ar AFTER INSERT ROW on _hyper_11_17_chunk: ("Sun Dec 31 22:0
 SELECT count(*) FROM trigger_test;
  count 
 -------
-     3
+     5
 (1 row)
 
 ROLLBACK;
@@ -700,15 +695,15 @@ DROP TRIGGER t4_ar ON trigger_test;
 CREATE CONSTRAINT TRIGGER t4_constraint AFTER INSERT ON trigger_test FOR EACH ROW EXECUTE FUNCTION stmt_trig_info();
 \set ON_ERROR_STOP 0
 INSERT INTO trigger_test SELECT '2000-01-01',1,0;
-ERROR:  after insert row trigger on compressed chunk not supported
+NOTICE:  Trigger t4_constraint AFTER INSERT ROW on _hyper_11_15_chunk: ("Sat Jan 01 00:00:00 2000 PST",1,0) <NULL>
 COPY trigger_test FROM STDIN DELIMITER ',';
-ERROR:  after insert row trigger on compressed chunk not supported
+NOTICE:  Trigger t4_constraint AFTER INSERT ROW on _hyper_11_15_chunk: ("Fri Dec 31 22:00:00 1999 PST",1,0) <NULL>
 \set ON_ERROR_STOP 1
 -- should not insert rows. count is 1
 SELECT count(*) FROM trigger_test;
  count 
 -------
-     1
+     5
 (1 row)
 
 DROP trigger t4_constraint ON trigger_test;
@@ -730,7 +725,7 @@ CREATE TRIGGER t1_mod BEFORE INSERT ON trigger_test FOR EACH ROW EXECUTE FUNCTIO
 SELECT count(*) FROM trigger_test;
  count 
 -------
-     2
+     6
 (1 row)
 
 BEGIN;
@@ -742,11 +737,15 @@ NOTICE:  Trigger t1_mod BEFORE INSERT ROW on _hyper_11_18_chunk: ("Fri Jan 01 00
 SELECT * FROM trigger_test ORDER BY 1 ,2, 5;
              time             | device | value | addcolv | addcoli 
 ------------------------------+--------+-------+---------+---------
+ Fri Dec 31 22:00:00 1999 PST |      1 |     0 |         |        
+ Fri Dec 31 22:00:00 1999 PST |      1 |     0 |         |        
  Sat Jan 01 00:00:00 2000 PST |      1 |   111 | eleven  |     111
  Sat Jan 01 00:00:00 2000 PST |      1 |     1 |         |        
+ Sat Jan 01 00:00:00 2000 PST |      1 |     0 |         |        
+ Sat Jan 01 00:00:00 2000 PST |      1 |     0 |         |        
  Fri Jan 01 00:00:00 2010 PST |     10 |    10 | ten     |     222
  Fri Jan 01 00:00:00 2010 PST |     10 |   110 | ten     |     222
-(4 rows)
+(8 rows)
 
 ROLLBACK;
 DROP TABLE trigger_test;
@@ -791,27 +790,32 @@ SELECT compress_chunk(format('%I.%I',chunk_schema,chunk_name), true) FROM timesc
 INSERT INTO test_ordering SELECT 1;
 -- should not be ordered append
 :PREFIX SELECT * FROM test_ordering ORDER BY 1;
-                        QUERY PLAN                         
------------------------------------------------------------
+                           QUERY PLAN                            
+-----------------------------------------------------------------
  Sort
    Sort Key: _hyper_13_20_chunk."time"
-   ->  Custom Scan (DecompressChunk) on _hyper_13_20_chunk
-         ->  Seq Scan on compress_hyper_14_21_chunk
-(4 rows)
+   ->  Append
+         ->  Custom Scan (DecompressChunk) on _hyper_13_20_chunk
+               ->  Seq Scan on compress_hyper_14_21_chunk
+         ->  Seq Scan on _hyper_13_20_chunk
+(6 rows)
 
 INSERT INTO test_ordering VALUES (105),(104),(103);
 -- should be ordered append
 :PREFIX SELECT * FROM test_ordering ORDER BY 1;
                                               QUERY PLAN                                              
 ------------------------------------------------------------------------------------------------------
- Custom Scan (ChunkAppend) on test_ordering
-   Order: test_ordering."time"
+ Merge Append
+   Sort Key: _hyper_13_20_chunk."time"
    ->  Sort
          Sort Key: _hyper_13_20_chunk."time"
          ->  Custom Scan (DecompressChunk) on _hyper_13_20_chunk
                ->  Seq Scan on compress_hyper_14_21_chunk
+   ->  Sort
+         Sort Key: _hyper_13_20_chunk."time"
+         ->  Seq Scan on _hyper_13_20_chunk
    ->  Index Only Scan Backward using _hyper_13_22_chunk_test_ordering_time_idx on _hyper_13_22_chunk
-(7 rows)
+(10 rows)
 
 --insert into compressed + uncompressed chunk
 INSERT INTO test_ordering VALUES (21), (22),(113);
@@ -844,17 +848,20 @@ NOTICE:  chunk "_hyper_13_20_chunk" is already compressed
 :PREFIX SELECT * FROM test_ordering ORDER BY 1;
                                   QUERY PLAN                                   
 -------------------------------------------------------------------------------
- Custom Scan (ChunkAppend) on test_ordering
-   Order: test_ordering."time"
+ Merge Append
+   Sort Key: _hyper_13_20_chunk."time"
    ->  Sort
          Sort Key: _hyper_13_20_chunk."time"
          ->  Custom Scan (DecompressChunk) on _hyper_13_20_chunk
                ->  Seq Scan on compress_hyper_14_21_chunk
+   ->  Sort
+         Sort Key: _hyper_13_20_chunk."time"
+         ->  Seq Scan on _hyper_13_20_chunk
    ->  Custom Scan (DecompressChunk) on _hyper_13_22_chunk
          ->  Sort
                Sort Key: compress_hyper_14_23_chunk._ts_meta_sequence_num DESC
                ->  Seq Scan on compress_hyper_14_23_chunk
-(10 rows)
+(13 rows)
 
 -- TEST cagg triggers with insert into compressed chunk
 CREATE TABLE conditions (
diff --git a/tsl/test/expected/compression_insert-15.out b/tsl/test/expected/compression_insert-15.out
index 2b2b00f89..8e2ee9321 100644
--- a/tsl/test/expected/compression_insert-15.out
+++ b/tsl/test/expected/compression_insert-15.out
@@ -40,7 +40,7 @@ INSERT INTO test1 SELECT '2020-01-02 11:16:00-05' , 11, 16, 'new' ;
 SELECT COUNT(*) from _timescaledb_internal.compress_hyper_2_2_chunk;
  count 
 -------
-     2
+     1
 (1 row)
 
 INSERT INTO test1 SELECT '2020-01-02 11:16:00-05' , i, i +5, 'clay'
@@ -48,7 +48,7 @@ FROM (Select generate_series(10, 20, 1) i ) q;
 SELECT COUNT(*) from _timescaledb_internal.compress_hyper_2_2_chunk;
  count 
 -------
-    13
+     1
 (1 row)
 
 SELECT count(*) from test1;
@@ -62,7 +62,7 @@ COPY test1 FROM STDIN DELIMITER ',';
 SELECT COUNT(*) from _timescaledb_internal.compress_hyper_2_2_chunk;
  count 
 -------
-    14
+     1
 (1 row)
 
 -- multi row copy
@@ -70,7 +70,7 @@ COPY test1 FROM STDIN DELIMITER ',';
 SELECT COUNT(*) from _timescaledb_internal.compress_hyper_2_2_chunk;
  count 
 -------
-    17
+     1
 (1 row)
 
 --Verify that all the data went into the initial chunk
@@ -151,7 +151,7 @@ ALTER TABLE test1 ADD COLUMN newtcol varchar(400);
 SELECT count(*) from _timescaledb_internal.compress_hyper_2_2_chunk;
  count 
 -------
-    19
+     1
 (1 row)
 
 INSERT INTO test1 SELECT '2020-01-02 11:16:00-05' , 100, 101, 'prev101', 'this is the newtcol101';
@@ -176,7 +176,7 @@ SELECT * FROM test1 WHERE b = 16 order by 1, 2, 3, 4, 5;
 SELECT count(*) from _timescaledb_internal.compress_hyper_2_2_chunk;
  count 
 -------
-    26
+     1
 (1 row)
 
 SELECT count(*)
@@ -192,7 +192,7 @@ COPY test1 FROM STDIN DELIMITER ',';
 SELECT count(*) from _timescaledb_internal.compress_hyper_2_2_chunk;
  count 
 -------
-    30
+     1
 (1 row)
 
 SELECT count(*)
@@ -665,21 +665,16 @@ CREATE TRIGGER t4_ar AFTER INSERT ON trigger_test FOR EACH ROW EXECUTE FUNCTION
 \set ON_ERROR_STOP 0
 \set VERBOSITY default
 INSERT INTO trigger_test SELECT '2000-01-01',1,0;
-ERROR:  after insert row trigger on compressed chunk not supported
-DETAIL:  Triggers: t4_ar
-HINT:  Decompress the chunk first before inserting into it.
+NOTICE:  Trigger t4_ar AFTER INSERT ROW on _hyper_11_15_chunk: ("Sat Jan 01 00:00:00 2000 PST",1,0) <NULL>
 COPY trigger_test FROM STDIN DELIMITER ',';
-ERROR:  after insert row trigger on compressed chunk not supported
-DETAIL:  Triggers: t4_ar
-HINT:  Decompress the chunk first before inserting into it.
-CONTEXT:  COPY trigger_test, line 1: "2000-01-01 01:00:00-05,1,0"
+NOTICE:  Trigger t4_ar AFTER INSERT ROW on _hyper_11_15_chunk: ("Fri Dec 31 22:00:00 1999 PST",1,0) <NULL>
 \set VERBOSITY terse
 \set ON_ERROR_STOP 1
 -- should not insert rows. count is 1
 SELECT count(*) FROM trigger_test;
  count 
 -------
-     1
+     3
 (1 row)
 
 BEGIN;
@@ -691,7 +686,7 @@ NOTICE:  Trigger t4_ar AFTER INSERT ROW on _hyper_11_17_chunk: ("Sun Dec 31 22:0
 SELECT count(*) FROM trigger_test;
  count 
 -------
-     3
+     5
 (1 row)
 
 ROLLBACK;
@@ -700,15 +695,15 @@ DROP TRIGGER t4_ar ON trigger_test;
 CREATE CONSTRAINT TRIGGER t4_constraint AFTER INSERT ON trigger_test FOR EACH ROW EXECUTE FUNCTION stmt_trig_info();
 \set ON_ERROR_STOP 0
 INSERT INTO trigger_test SELECT '2000-01-01',1,0;
-ERROR:  after insert row trigger on compressed chunk not supported
+NOTICE:  Trigger t4_constraint AFTER INSERT ROW on _hyper_11_15_chunk: ("Sat Jan 01 00:00:00 2000 PST",1,0) <NULL>
 COPY trigger_test FROM STDIN DELIMITER ',';
-ERROR:  after insert row trigger on compressed chunk not supported
+NOTICE:  Trigger t4_constraint AFTER INSERT ROW on _hyper_11_15_chunk: ("Fri Dec 31 22:00:00 1999 PST",1,0) <NULL>
 \set ON_ERROR_STOP 1
 -- should not insert rows. count is 1
 SELECT count(*) FROM trigger_test;
  count 
 -------
-     1
+     5
 (1 row)
 
 DROP trigger t4_constraint ON trigger_test;
@@ -730,7 +725,7 @@ CREATE TRIGGER t1_mod BEFORE INSERT ON trigger_test FOR EACH ROW EXECUTE FUNCTIO
 SELECT count(*) FROM trigger_test;
  count 
 -------
-     2
+     6
 (1 row)
 
 BEGIN;
@@ -742,11 +737,15 @@ NOTICE:  Trigger t1_mod BEFORE INSERT ROW on _hyper_11_18_chunk: ("Fri Jan 01 00
 SELECT * FROM trigger_test ORDER BY 1 ,2, 5;
              time             | device | value | addcolv | addcoli 
 ------------------------------+--------+-------+---------+---------
+ Fri Dec 31 22:00:00 1999 PST |      1 |     0 |         |        
+ Fri Dec 31 22:00:00 1999 PST |      1 |     0 |         |        
  Sat Jan 01 00:00:00 2000 PST |      1 |   111 | eleven  |     111
  Sat Jan 01 00:00:00 2000 PST |      1 |     1 |         |        
+ Sat Jan 01 00:00:00 2000 PST |      1 |     0 |         |        
+ Sat Jan 01 00:00:00 2000 PST |      1 |     0 |         |        
  Fri Jan 01 00:00:00 2010 PST |     10 |    10 | ten     |     222
  Fri Jan 01 00:00:00 2010 PST |     10 |   110 | ten     |     222
-(4 rows)
+(8 rows)
 
 ROLLBACK;
 DROP TABLE trigger_test;
@@ -791,27 +790,32 @@ SELECT compress_chunk(format('%I.%I',chunk_schema,chunk_name), true) FROM timesc
 INSERT INTO test_ordering SELECT 1;
 -- should not be ordered append
 :PREFIX SELECT * FROM test_ordering ORDER BY 1;
-                        QUERY PLAN                         
------------------------------------------------------------
+                           QUERY PLAN                            
+-----------------------------------------------------------------
  Sort
    Sort Key: _hyper_13_20_chunk."time"
-   ->  Custom Scan (DecompressChunk) on _hyper_13_20_chunk
-         ->  Seq Scan on compress_hyper_14_21_chunk
-(4 rows)
+   ->  Append
+         ->  Custom Scan (DecompressChunk) on _hyper_13_20_chunk
+               ->  Seq Scan on compress_hyper_14_21_chunk
+         ->  Seq Scan on _hyper_13_20_chunk
+(6 rows)
 
 INSERT INTO test_ordering VALUES (105),(104),(103);
 -- should be ordered append
 :PREFIX SELECT * FROM test_ordering ORDER BY 1;
                                               QUERY PLAN                                              
 ------------------------------------------------------------------------------------------------------
- Custom Scan (ChunkAppend) on test_ordering
-   Order: test_ordering."time"
+ Merge Append
+   Sort Key: _hyper_13_20_chunk."time"
    ->  Sort
          Sort Key: _hyper_13_20_chunk."time"
          ->  Custom Scan (DecompressChunk) on _hyper_13_20_chunk
                ->  Seq Scan on compress_hyper_14_21_chunk
+   ->  Sort
+         Sort Key: _hyper_13_20_chunk."time"
+         ->  Seq Scan on _hyper_13_20_chunk
    ->  Index Only Scan Backward using _hyper_13_22_chunk_test_ordering_time_idx on _hyper_13_22_chunk
-(7 rows)
+(10 rows)
 
 --insert into compressed + uncompressed chunk
 INSERT INTO test_ordering VALUES (21), (22),(113);
@@ -844,17 +848,20 @@ NOTICE:  chunk "_hyper_13_20_chunk" is already compressed
 :PREFIX SELECT * FROM test_ordering ORDER BY 1;
                                   QUERY PLAN                                   
 -------------------------------------------------------------------------------
- Custom Scan (ChunkAppend) on test_ordering
-   Order: test_ordering."time"
+ Merge Append
+   Sort Key: _hyper_13_20_chunk."time"
    ->  Sort
          Sort Key: _hyper_13_20_chunk."time"
          ->  Custom Scan (DecompressChunk) on _hyper_13_20_chunk
                ->  Seq Scan on compress_hyper_14_21_chunk
+   ->  Sort
+         Sort Key: _hyper_13_20_chunk."time"
+         ->  Seq Scan on _hyper_13_20_chunk
    ->  Custom Scan (DecompressChunk) on _hyper_13_22_chunk
          ->  Sort
                Sort Key: compress_hyper_14_23_chunk._ts_meta_sequence_num DESC
                ->  Seq Scan on compress_hyper_14_23_chunk
-(10 rows)
+(13 rows)
 
 -- TEST cagg triggers with insert into compressed chunk
 CREATE TABLE conditions (
diff --git a/tsl/test/expected/dist_compression.out b/tsl/test/expected/dist_compression.out
index e1fdb767e..d7916c47e 100644
--- a/tsl/test/expected/dist_compression.out
+++ b/tsl/test/expected/dist_compression.out
@@ -931,7 +931,7 @@ SELECT count(*) from test_recomp_int where val = 10;
 SELECT * from test_recomp_int_chunk_status ORDER BY 1;
        chunk_name       | chunk_status 
 ------------------------+--------------
- _dist_hyper_6_24_chunk |            3
+ _dist_hyper_6_24_chunk |            9
 (1 row)
 
 SELECT
@@ -961,7 +961,7 @@ insert into test_recomp_int select generate_series(5,7), 10;
 SELECT * from test_recomp_int_chunk_status ORDER BY 1;
        chunk_name       | chunk_status 
 ------------------------+--------------
- _dist_hyper_6_24_chunk |            3
+ _dist_hyper_6_24_chunk |            9
 (1 row)
 
 --run the compression policy job, it will recompress chunks that are unordered
@@ -1004,7 +1004,7 @@ INSERT INTO test_recomp_int VALUES( 65, 10);
 SELECT * from test_recomp_int_chunk_status ORDER BY 1;
        chunk_name       | chunk_status 
 ------------------------+--------------
- _dist_hyper_6_24_chunk |            3
+ _dist_hyper_6_24_chunk |            9
  _dist_hyper_6_25_chunk |            0
  _dist_hyper_6_26_chunk |            0
 (3 rows)
@@ -1024,7 +1024,7 @@ ORDER BY chunk;
 SELECT * from test_recomp_int_chunk_status ORDER BY 1;
        chunk_name       | chunk_status 
 ------------------------+--------------
- _dist_hyper_6_24_chunk |            3
+ _dist_hyper_6_24_chunk |            9
  _dist_hyper_6_25_chunk |            1
  _dist_hyper_6_26_chunk |            1
 (3 rows)
@@ -1034,9 +1034,9 @@ COPY test_recomp_int  FROM STDIN WITH DELIMITER ',';
 SELECT * from test_recomp_int_chunk_status ORDER BY 1;
        chunk_name       | chunk_status 
 ------------------------+--------------
- _dist_hyper_6_24_chunk |            3
- _dist_hyper_6_25_chunk |            3
- _dist_hyper_6_26_chunk |            3
+ _dist_hyper_6_24_chunk |            9
+ _dist_hyper_6_25_chunk |            9
+ _dist_hyper_6_26_chunk |            9
 (3 rows)
 
 SELECT time_bucket(20, time), count(*)
@@ -1069,7 +1069,7 @@ SELECT * from test_recomp_int_chunk_status ORDER BY 1;
 ------------------------+--------------
  _dist_hyper_6_24_chunk |            1
  _dist_hyper_6_25_chunk |            1
- _dist_hyper_6_26_chunk |            3
+ _dist_hyper_6_26_chunk |            9
 (3 rows)
 
 CALL run_job(:compressjob_id);
@@ -1225,7 +1225,7 @@ FROM _timescaledb_catalog.hypertable h, _timescaledb_catalog.chunk c
 WHERE h.id = c.hypertable_id and h.table_name = 'test_recomp_int' ORDER BY 1;
        chunk_name       | chunk_status | dropped | comp_id 
 ------------------------+--------------+---------+---------
- _dist_hyper_6_24_chunk |            3 | f       |        
+ _dist_hyper_6_24_chunk |            9 | f       |        
  _dist_hyper_6_25_chunk |            1 | f       |        
  _dist_hyper_6_26_chunk |            1 | f       |        
  _dist_hyper_6_28_chunk |            0 | f       |        
diff --git a/tsl/test/isolation/expected/compression_ddl_iso.out b/tsl/test/isolation/expected/compression_ddl_iso.out
index 29398a53b..d22bc44d4 100644
--- a/tsl/test/isolation/expected/compression_ddl_iso.out
+++ b/tsl/test/isolation/expected/compression_ddl_iso.out
@@ -327,7 +327,7 @@ step SChunkStat: SELECT status from _timescaledb_catalog.chunk
        WHERE id = ( select min(ch.id) FROM _timescaledb_catalog.hypertable ht, _timescaledb_catalog.chunk ch WHERE ch.hypertable_id = ht.id AND ht.table_name like 'ts_device_table');
 status
 ------
-     3
+     9
 (1 row)
 
 
@@ -353,7 +353,7 @@ step SChunkStat: SELECT status from _timescaledb_catalog.chunk
        WHERE id = ( select min(ch.id) FROM _timescaledb_catalog.hypertable ht, _timescaledb_catalog.chunk ch WHERE ch.hypertable_id = ht.id AND ht.table_name like 'ts_device_table');
 status
 ------
-     3
+     9
 (1 row)
 
 step LockChunk1: